From 40cc50cff4b2ecc762463ca6ec3856b1e6c7be68 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Mon, 24 Mar 2025 06:58:49 +0200 Subject: [PATCH 01/83] sdks/python: enrich data with CloudSQL --- .../enrichment_handlers/cloudsql.py | 169 ++++++++++++++++++ 1 file changed, 169 insertions(+) create mode 100644 sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py new file mode 100644 index 000000000000..a1a19d664777 --- /dev/null +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py @@ -0,0 +1,169 @@ +# +# 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 logging +from collections.abc import Callable +from enum import Enum +from typing import Any +from typing import Optional + +from google.cloud.sql.connector import Connector + +import apache_beam as beam +from apache_beam.transforms.enrichment import EnrichmentSourceHandler +from apache_beam.transforms.enrichment_handlers.utils import ExceptionLevel + +__all__ = [ + 'CloudSQLEnrichmentHandler', +] + +# RowKeyFn takes beam.Row and returns tuple of (key_id, key_value). +RowKeyFn = Callable[[beam.Row], tuple[str]] + +_LOGGER = logging.getLogger(__name__) + + +class DatabaseTypeAdapter(Enum): + POSTGRESQL = "pg8000" + MYSQL = "pymysql" + SQLSERVER = "pytds" + + +class CloudSQLEnrichmentHandler(EnrichmentSourceHandler[beam.Row, beam.Row]): + """A handler for :class:`apache_beam.transforms.enrichment.Enrichment` + transform to interact with Google Cloud SQL databases. + + Args: + project_id (str): GCP project-id of the Cloud SQL instance. + region_id (str): GCP region-id of the Cloud SQL instance. + instance_id (str): GCP instance-id of the Cloud SQL instance. + database_type_adapter (DatabaseTypeAdapter): The type of database adapter to use. + Supported adapters are: POSTGRESQL (pg8000), MYSQL (pymysql), and SQLSERVER (pytds). + database_id (str): The id of the database to connect to. + database_user (str): The username for connecting to the database. + database_password (str): The password for connecting to the database. + table_id (str): The name of the table to query. + row_key (str): Field name from the input `beam.Row` object to use as + identifier for database querying. + row_key_fn: A lambda function that returns a string key from the + input row. Used to build/extract the identifier for the database query. + exception_level: A `enum.Enum` value from + ``apache_beam.transforms.enrichment_handlers.utils.ExceptionLevel`` + to set the level when no matching record is found from the database query. + Defaults to ``ExceptionLevel.WARN``. + """ + def __init__( + self, + region_id: str, + project_id: str, + instance_id: str, + database_type_adapter: DatabaseTypeAdapter, + database_id: str, + database_user: str, + database_password: str, + table_id: str, + row_key: str = "", + *, + row_key_fn: Optional[RowKeyFn] = None, + exception_level: ExceptionLevel = ExceptionLevel.WARN, + ): + self._project_id = project_id + self._region_id = region_id + self._instance_id = instance_id + self._database_type_adapter = database_type_adapter + self._database_id = database_id + self._database_user = database_user + self._database_password = database_password + self._table_id = table_id + self._row_key = row_key + self._row_key_fn = row_key_fn + self._exception_level = exception_level + if ((not self._row_key_fn and not self._row_key) or + bool(self._row_key_fn and self._row_key)): + raise ValueError( + "Please specify exactly one of `row_key` or a lambda " + "function with `row_key_fn` to extract the row key " + "from the input row.") + + def __enter__(self): + """Connect to the the Cloud SQL instance.""" + self.connector = Connector() + self.client = self.connector.connect( + f"{self._project_id}:{self._region_id}:{self._instance_id}", + driver=self._database_type_adapter.value, + db=self._database_id, + user=self._database_user, + password=self._database_password, + ) + self.cursor = self.client.cursor() + + def __call__(self, request: beam.Row, *args, **kwargs): + """ + Executes a query to the Cloud SQL instance and returns + a `Tuple` of request and response. + + Args: + request: the input `beam.Row` to enrich. + """ + response_dict: dict[str, Any] = {} + row_key_str: str = "" + + try: + if self._row_key_fn: + self._row_key, row_key = self._row_key_fn(request) + else: + request_dict = request._asdict() + row_key_str = str(request_dict[self._row_key]) + row_key = row_key_str + + query = f"SELECT * FROM {self._table_id} WHERE {self._row_key} = %s" + self.cursor.execute(query, (row_key, )) + result = self.cursor.fetchone() + + if result: + columns = [col[0] for col in self.cursor.description] + for i, value in enumerate(result): + response_dict[columns[i]] = value + elif self._exception_level == ExceptionLevel.WARN: + _LOGGER.warning( + 'No matching record found for row_key: %s in table: %s', + row_key_str, + self._table_id) + elif self._exception_level == ExceptionLevel.RAISE: + raise ValueError( + 'No matching record found for row_key: %s in table: %s' % + (row_key_str, self._table_id)) + except KeyError: + raise KeyError('row_key %s not found in input PCollection.' % row_key_str) + except Exception as e: + raise e + + return request, beam.Row(**response_dict) + + def __exit__(self, exc_type, exc_val, exc_tb): + """Clean the instantiated Cloud SQL client.""" + self.cursor.close() + self.client.close() + self.connector.close() + self.cursor, self.client, self.connector = None, None, None + + def get_cache_key(self, request: beam.Row) -> str: + """Returns a string formatted with row key since it is unique to + a request made to the Cloud SQL instance.""" + if self._row_key_fn: + id, value = self._row_key_fn(request) + return f"{id}: {value}" + return f"{self._row_key}: {request._asdict()[self._row_key]}" From 507d3ec5461965783fd89363b431a4a862de918e Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Mon, 24 Mar 2025 21:15:47 +0200 Subject: [PATCH 02/83] sdks/python: unit test `CloudSQLEnrichmentHandler` --- .../enrichment_handlers/cloudsql_test.py | 46 +++++++++++++++++++ 1 file changed, 46 insertions(+) create mode 100644 sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_test.py diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_test.py new file mode 100644 index 000000000000..4f2547b67fee --- /dev/null +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_test.py @@ -0,0 +1,46 @@ +# +# 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 parameterized import parameterized + +try: + from apache_beam.transforms.enrichment_handlers.cloudsql import CloudSQLEnrichmentHandler, DatabaseTypeAdapter + from apache_beam.transforms.enrichment_handlers.cloudsql_it_test import _row_key_fn +except ImportError: + raise unittest.SkipTest('Cloud SQL test dependencies are not installed.') + + +class TestCloudSQLEnrichmentHandler(unittest.TestCase): + @parameterized.expand([('product_id', _row_key_fn), ('', None)]) + def test_cloud_sql_enrichment_invalid_args(self, row_key, row_key_fn): + with self.assertRaises(ValueError): + _ = CloudSQLEnrichmentHandler( + project_id='apache-beam-testing', + region_id='us-east1', + instance_id='beam-test', + table_id='cloudsql-enrichment-test', + database_type_adapter=DatabaseTypeAdapter.POSTGRESQL, + database_id='', + database_user='', + database_password='', + row_key=row_key, + row_key_fn=row_key_fn) + + +if __name__ == '__main__': + unittest.main() From b4c11d3a93b177652c0b86b344ab22773346ecf1 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Tue, 25 Mar 2025 23:19:09 +0200 Subject: [PATCH 03/83] sdks/python: itest `CloudSQLEnrichmentHandler` --- .../enrichment_handlers/cloudsql_it_test.py | 396 ++++++++++++++++++ 1 file changed, 396 insertions(+) create mode 100644 sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py new file mode 100644 index 000000000000..63b9ee2d17b9 --- /dev/null +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py @@ -0,0 +1,396 @@ +# +# 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 logging +import unittest +from unittest.mock import MagicMock +import pytest +import apache_beam as beam +from apache_beam.coders import coders +from apache_beam.testing.test_pipeline import TestPipeline +from apache_beam.testing.util import BeamAssertException +from apache_beam.transforms.enrichment import Enrichment +from apache_beam.transforms.enrichment_handlers.cloudsql import ( + CloudSQLEnrichmentHandler, + DatabaseTypeAdapter, + ExceptionLevel, +) +from testcontainers.redis import RedisContainer +from google.cloud.sql.connector import Connector +import os + +_LOGGER = logging.getLogger(__name__) + + +def _row_key_fn(request: beam.Row, key_id="product_id") -> tuple[str]: + key_value = str(getattr(request, key_id)) + return (key_id, key_value) + + +class ValidateResponse(beam.DoFn): + """ValidateResponse validates if a PCollection of `beam.Row` + has the required fields.""" + def __init__( + self, + n_fields: int, + fields: list[str], + enriched_fields: dict[str, list[str]], + ): + self.n_fields = n_fields + self._fields = fields + self._enriched_fields = enriched_fields + + def process(self, element: beam.Row, *args, **kwargs): + element_dict = element.as_dict() + if len(element_dict.keys()) != self.n_fields: + raise BeamAssertException( + "Expected %d fields in enriched PCollection:" % self.n_fields) + + for field in self._fields: + if field not in element_dict or element_dict[field] is None: + raise BeamAssertException(f"Expected a not None field: {field}") + + for key in self._enriched_fields: + if key not in element_dict: + raise BeamAssertException( + f"Response from Cloud SQL should contain {key} column.") + + +def create_rows(cursor): + """Insert test rows into the Cloud SQL database table.""" + cursor.execute( + """ + CREATE TABLE IF NOT EXISTS products ( + product_id SERIAL PRIMARY KEY, + product_name VARCHAR(255), + product_stock INT + ) + """) + cursor.execute( + """ + INSERT INTO products (product_name, product_stock) + VALUES + ('pixel 5', 2), + ('pixel 6', 4), + ('pixel 7', 20), + ('pixel 8', 10), + ('iphone 11', 3), + ('iphone 12', 7), + ('iphone 13', 8), + ('iphone 14', 3) + ON CONFLICT DO NOTHING + """) + + +@pytest.mark.uses_testcontainer +class TestCloudSQLEnrichment(unittest.TestCase): + @classmethod + def setUpClass(cls): + cls.project_id = "apache-beam-testing" + cls.region_id = "us-central1" + cls.instance_id = "beam-test" + cls.database_id = "postgres" + cls.database_user = os.getenv("BEAM_TEST_CLOUDSQL_PG_USER") + cls.database_password = os.getenv("BEAM_TEST_CLOUDSQL_PG_PASSWORD") + cls.table_id = "products" + cls.row_key = "product_id" + cls.database_type_adapter = DatabaseTypeAdapter.POSTGRESQL + cls.req = [ + beam.Row(sale_id=1, customer_id=1, product_id=1, quantity=1), + beam.Row(sale_id=3, customer_id=3, product_id=2, quantity=3), + beam.Row(sale_id=5, customer_id=5, product_id=3, quantity=2), + beam.Row(sale_id=7, customer_id=7, product_id=4, quantity=1), + ] + cls.connector = Connector() + cls.client = cls.connector.connect( + f"{cls.project_id}:{cls.region_id}:{cls.instance_id}", + driver=cls.database_type_adapter.value, + db=cls.database_id, + user=cls.database_user, + password=cls.database_password, + ) + cls.cursor = cls.client.cursor() + create_rows(cls.cursor) + cls.cache_client_retries = 3 + + def _start_cache_container(self): + for i in range(self.cache_client_retries): + try: + self.container = RedisContainer(image="redis:7.2.4") + self.container.start() + self.host = self.container.get_container_host_ip() + self.port = self.container.get_exposed_port(6379) + self.cache_client = self.container.get_client() + break + except Exception as e: + if i == self.cache_client_retries - 1: + _LOGGER.error( + f"Unable to start redis container for RRIO tests after {self.cache_client_retries} retries." + ) + raise e + + @classmethod + def tearDownClass(cls): + cls.cursor.close() + cls.client.close() + cls.connector.close() + cls.cursor, cls.client, cls.connector = None, None, None + + def test_enrichment_with_cloudsql(self): + expected_fields = [ + "sale_id", + "customer_id", + "product_id", + "quantity", + "product_name", + "product_stock", + ] + expected_enriched_fields = ["product_id", "product_name", "product_stock"] + cloudsql = CloudSQLEnrichmentHandler( + region_id=self.region_id, + project_id=self.project_id, + instance_id=self.instance_id, + database_type_adapter=self.database_type_adapter, + database_id=self.database_id, + database_user=self.database_user, + database_password=self.database_password, + table_id=self.table_id, + row_key=self.row_key, + ) + with TestPipeline(is_integration_test=True) as test_pipeline: + _ = ( + test_pipeline + | "Create" >> beam.Create(self.req) + | "Enrich W/ CloudSQL" >> Enrichment(cloudsql) + | "Validate Response" >> beam.ParDo( + ValidateResponse( + len(expected_fields), + expected_fields, + expected_enriched_fields, + ))) + + def test_enrichment_with_cloudsql_no_enrichment(self): + expected_fields = ["sale_id", "customer_id", "product_id", "quantity"] + expected_enriched_fields = {} + cloudsql = CloudSQLEnrichmentHandler( + region_id=self.region_id, + project_id=self.project_id, + instance_id=self.instance_id, + database_type_adapter=self.database_type_adapter, + database_id=self.database_id, + database_user=self.database_user, + database_password=self.database_password, + table_id=self.table_id, + row_key=self.row_key, + ) + req = [beam.Row(sale_id=1, customer_id=1, product_id=99, quantity=1)] + with TestPipeline(is_integration_test=True) as test_pipeline: + _ = ( + test_pipeline + | "Create" >> beam.Create(req) + | "Enrich W/ CloudSQL" >> Enrichment(cloudsql) + | "Validate Response" >> beam.ParDo( + ValidateResponse( + len(expected_fields), + expected_fields, + expected_enriched_fields, + ))) + + def test_enrichment_with_cloudsql_raises_key_error(self): + cloudsql = CloudSQLEnrichmentHandler( + region_id=self.region_id, + project_id=self.project_id, + instance_id=self.instance_id, + database_type_adapter=self.database_type_adapter, + database_id=self.database_id, + database_user=self.database_user, + database_password=self.database_password, + table_id=self.table_id, + row_key="car_name", + ) + with self.assertRaises(KeyError): + test_pipeline = TestPipeline() + _ = ( + test_pipeline + | "Create" >> beam.Create(self.req) + | "Enrich W/ CloudSQL" >> Enrichment(cloudsql)) + res = test_pipeline.run() + res.wait_until_finish() + + def test_enrichment_with_cloudsql_raises_not_found(self): + """Raises a database error when the GCP Cloud SQL table doesn't exist.""" + table_id = "invalid_table" + cloudsql = CloudSQLEnrichmentHandler( + region_id=self.region_id, + project_id=self.project_id, + instance_id=self.instance_id, + database_type_adapter=self.database_type_adapter, + database_id=self.database_id, + database_user=self.database_user, + database_password=self.database_password, + table_id=table_id, + row_key=self.row_key, + ) + try: + test_pipeline = beam.Pipeline() + _ = ( + test_pipeline + | "Create" >> beam.Create(self.req) + | "Enrich W/ CloudSQL" >> Enrichment(cloudsql)) + res = test_pipeline.run() + res.wait_until_finish() + except (PgDatabaseError, RuntimeError) as e: + self.assertIn(f'relation "{table_id}" does not exist', str(e)) + + def test_enrichment_with_cloudsql_exception_level(self): + """raises a `ValueError` exception when the GCP Cloud SQL query returns + an empty row.""" + cloudsql = CloudSQLEnrichmentHandler( + region_id=self.region_id, + project_id=self.project_id, + instance_id=self.instance_id, + database_type_adapter=self.database_type_adapter, + database_id=self.database_id, + database_user=self.database_user, + database_password=self.database_password, + table_id=self.table_id, + row_key=self.row_key, + exception_level=ExceptionLevel.RAISE, + ) + req = [beam.Row(sale_id=1, customer_id=1, product_id=11, quantity=1)] + with self.assertRaises(ValueError): + test_pipeline = beam.Pipeline() + _ = ( + test_pipeline + | "Create" >> beam.Create(req) + | "Enrich W/ CloudSQL" >> Enrichment(cloudsql)) + res = test_pipeline.run() + res.wait_until_finish() + + def test_cloudsql_enrichment_with_lambda(self): + expected_fields = [ + "sale_id", + "customer_id", + "product_id", + "quantity", + "product_name", + "product_stock", + ] + expected_enriched_fields = ["product_id", "product_name", "product_stock"] + cloudsql = CloudSQLEnrichmentHandler( + region_id=self.region_id, + project_id=self.project_id, + instance_id=self.instance_id, + database_type_adapter=self.database_type_adapter, + database_id=self.database_id, + database_user=self.database_user, + database_password=self.database_password, + table_id=self.table_id, + row_key_fn=_row_key_fn, + ) + with TestPipeline(is_integration_test=True) as test_pipeline: + _ = ( + test_pipeline + | "Create" >> beam.Create(self.req) + | "Enrich W/ CloudSQL" >> Enrichment(cloudsql) + | "Validate Response" >> beam.ParDo( + ValidateResponse( + len(expected_fields), + expected_fields, + expected_enriched_fields))) + + @pytest.fixture + def cache_container(self): + # Setup phase: start the container. + self._start_cache_container() + + # Hand control to the test. + yield + + # Cleanup phase: stop the container. It runs after the test completion + # even if it failed. + self.container.stop() + self.container = None + + @pytest.mark.usefixtures("cache_container") + def test_cloudsql_enrichment_with_redis(self): + expected_fields = [ + "sale_id", + "customer_id", + "product_id", + "quantity", + "product_name", + "product_stock", + ] + expected_enriched_fields = ["product_id", "product_name", "product_stock"] + cloudsql = CloudSQLEnrichmentHandler( + region_id=self.region_id, + project_id=self.project_id, + instance_id=self.instance_id, + database_type_adapter=self.database_type_adapter, + database_id=self.database_id, + database_user=self.database_user, + database_password=self.database_password, + table_id=self.table_id, + row_key_fn=_row_key_fn, + ) + with TestPipeline(is_integration_test=True) as test_pipeline: + _ = ( + test_pipeline + | "Create1" >> beam.Create(self.req) + | "Enrich W/ CloudSQL1" >> Enrichment(cloudsql).with_redis_cache( + self.host, self.port, 300) + | "Validate Response" >> beam.ParDo( + ValidateResponse( + len(expected_fields), + expected_fields, + expected_enriched_fields, + ))) + + # Manually check cache entry to verify entries were correctly stored. + c = coders.StrUtf8Coder() + for req in self.req: + key = cloudsql.get_cache_key(req) + response = self.cache_client.get(c.encode(key)) + if not response: + raise ValueError("No cache entry found for %s" % key) + + # Mock the CloudSQL handler to avoid actual database calls. + # This simulates a cache hit scenario by returning predefined data. + actual = CloudSQLEnrichmentHandler.__call__ + CloudSQLEnrichmentHandler.__call__ = MagicMock( + return_value=( + beam.Row(sale_id=1, customer_id=1, product_id=1, quantity=1), + beam.Row(), + )) + + # Run a second pipeline to verify cache is being used. + with TestPipeline(is_integration_test=True) as test_pipeline: + _ = ( + test_pipeline + | "Create2" >> beam.Create(self.req) + | "Enrich W/ CloudSQL2" >> Enrichment(cloudsql).with_redis_cache( + self.host, self.port) + | "Validate Response" >> beam.ParDo( + ValidateResponse( + len(expected_fields), + expected_fields, + expected_enriched_fields))) + CloudSQLEnrichmentHandler.__call__ = actual + + +if __name__ == "__main__": + unittest.main() From 43943b1adef6a9ddcbdce245ab416ff4a853679f Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Wed, 26 Mar 2025 00:59:47 +0200 Subject: [PATCH 04/83] website+sdks: doc `CloudSQLEnrichmentHandler` --- .../transforms/elementwise/enrichment.py | 40 +++++++++++++++++++ .../transforms/elementwise/enrichment_test.py | 14 +++++++ .../python/elementwise/enrichment.md | 1 + 3 files changed, 55 insertions(+) diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py index acee633b6f67..c88696bbc24d 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py @@ -50,6 +50,46 @@ def enrichment_with_bigtable(): | "Print" >> beam.Map(print)) # [END enrichment_with_bigtable] +def enrichment_with_cloudsql(): + # [START enrichment_with_cloudsql] + import apache_beam as beam + from apache_beam.transforms.enrichment import Enrichment + from apache_beam.transforms.enrichment_handlers.cloudsql import CloudSQLEnrichmentHandler, DatabaseTypeAdapter + import os + + project_id = 'apache-beam-testing' + region_id = 'us-east1' + instance_id = 'beam-test' + table_id = 'cloudsql-enrichment-test' + database_id = 'test-database' + database_user = os.getenv("BEAM_TEST_CLOUDSQL_PG_USER") + database_password = os.getenv("BEAM_TEST_CLOUDSQL_PG_PASSWORD") + row_key = 'product_id' + + data = [ + beam.Row(sale_id=1, customer_id=1, product_id=1, quantity=1), + beam.Row(sale_id=3, customer_id=3, product_id=2, quantity=3), + beam.Row(sale_id=5, customer_id=5, product_id=4, quantity=2), + ] + + cloudsql_handler = CloudSQLEnrichmentHandler( + project_id=project_id, + region_id=region_id, + instance_id=instance_id, + table_id=table_id, + database_type_adapter=DatabaseTypeAdapter.POSTGRESQL, + database_id=database_id, + database_user=database_user, + database_password=database_password, + row_key=row_key + ) + with beam.Pipeline() as p: + _ = ( + p + | "Create" >> beam.Create(data) + | "Enrich W/ CloudSQL" >> Enrichment(cloudsql_handler) + | "Print" >> beam.Map(print)) + # [END enrichment_with_cloudsql] def enrichment_with_vertex_ai(): # [START enrichment_with_vertex_ai] diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py index 8a7cdfbe9263..7086261c254e 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py @@ -41,6 +41,13 @@ def validate_enrichment_with_bigtable(): [END enrichment_with_bigtable]'''.splitlines()[1:-1] return expected +def validate_enrichment_with_cloudsql(): + expected = '''[START enrichment_with_cloudsql] +Row(sale_id=1, customer_id=1, product_id=1, quantity=1, product={'product_id': '1', 'product_name': 'pixel 5', 'product_stock': '2'}) +Row(sale_id=3, customer_id=3, product_id=2, quantity=3, product={'product_id': '2', 'product_name': 'pixel 6', 'product_stock': '4'}) +Row(sale_id=5, customer_id=5, product_id=4, quantity=2, product={'product_id': '4', 'product_name': 'pixel 8', 'product_stock': '10'}) + [END enrichment_with_cloudsql]'''.splitlines()[1:-1] + return expected def validate_enrichment_with_vertex_ai(): expected = '''[START enrichment_with_vertex_ai] @@ -68,6 +75,13 @@ def test_enrichment_with_bigtable(self, mock_stdout): expected = validate_enrichment_with_bigtable() self.assertEqual(output, expected) + def test_enrichment_with_cloudsql(self, mock_stdout): + from apache_beam.examples.snippets.transforms.elementwise.enrichment import enrichment_with_cloudsql + enrichment_with_cloudsql() + output = mock_stdout.getvalue().splitlines() + expected = validate_enrichment_with_cloudsql() + self.assertEqual(output, expected) + def test_enrichment_with_vertex_ai(self, mock_stdout): enrichment_with_vertex_ai() output = mock_stdout.getvalue().splitlines() diff --git a/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment.md b/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment.md index 6c05b6b515a4..0993963ec057 100644 --- a/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment.md +++ b/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment.md @@ -42,6 +42,7 @@ The following examples demonstrate how to create a pipeline that use the enrichm | Service | Example | |:-----------------------------------|:-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------| | Cloud Bigtable | [Enrichment with Bigtable](/documentation/transforms/python/elementwise/enrichment-bigtable/#example) | +| Cloud SQL | [Enrichment with CloudSQL](/documentation/transforms/python/elementwise/enrichment-cloudsql/#example) | | Vertex AI Feature Store | [Enrichment with Vertex AI Feature Store](/documentation/transforms/python/elementwise/enrichment-vertexai/#example-1-enrichment-with-vertex-ai-feature-store) | | Vertex AI Feature Store (Legacy) | [Enrichment with Legacy Vertex AI Feature Store](/documentation/transforms/python/elementwise/enrichment-vertexai/#example-2-enrichment-with-vertex-ai-feature-store-legacy) | {{< /table >}} From 846c30dbd7b9b6ed57fdd768460ad1c17ca39ee0 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Fri, 11 Apr 2025 14:10:03 +0200 Subject: [PATCH 05/83] sdks/python: address claudevdm feedback (1) --- .../enrichment_handlers/cloudsql.py | 382 +++++++++++++----- 1 file changed, 270 insertions(+), 112 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py index a1a19d664777..77ab9889c8da 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py @@ -14,156 +14,314 @@ # See the License for the specific language governing permissions and # limitations under the License. # -import logging from collections.abc import Callable +from collections.abc import Mapping from enum import Enum from typing import Any from typing import Optional +from typing import Union -from google.cloud.sql.connector import Connector +from sqlalchemy import create_engine, text import apache_beam as beam from apache_beam.transforms.enrichment import EnrichmentSourceHandler from apache_beam.transforms.enrichment_handlers.utils import ExceptionLevel -__all__ = [ - 'CloudSQLEnrichmentHandler', -] +QueryFn = Callable[[beam.Row], str] +ConditionValueFn = Callable[[beam.Row], list[Any]] -# RowKeyFn takes beam.Row and returns tuple of (key_id, key_value). -RowKeyFn = Callable[[beam.Row], tuple[str]] -_LOGGER = logging.getLogger(__name__) +def _validate_cloudsql_metadata( + table_id, + where_clause_template, + where_clause_fields, + where_clause_value_fn, + query_fn): + if query_fn: + if any([table_id, + where_clause_template, + where_clause_fields, + where_clause_value_fn]): + raise ValueError( + "Please provide either `query_fn` or the parameters `table_id`, " + "`where_clause_template`, and `where_clause_fields/where_clause_value_fn` " + "together.") + else: + if not (table_id and where_clause_template): + raise ValueError( + "Please provide either `query_fn` or the parameters " + "`table_id` and `where_clause_template` together.") + if (bool(where_clause_fields) == bool(where_clause_value_fn)): + raise ValueError( + "Please provide exactly one of `where_clause_fields` or " + "`where_clause_value_fn`.") class DatabaseTypeAdapter(Enum): - POSTGRESQL = "pg8000" + POSTGRESQL = "psycopg2" MYSQL = "pymysql" SQLSERVER = "pytds" + def to_sqlalchemy_dialect(self): + """ + Map the adapter type to its corresponding SQLAlchemy dialect. + Returns: + str: SQLAlchemy dialect string. + """ + if self == DatabaseTypeAdapter.POSTGRESQL: + return f"postgresql+{self.value}" + elif self == DatabaseTypeAdapter.MYSQL: + return f"mysql+{self.value}" + elif self == DatabaseTypeAdapter.SQLSERVER: + return f"mssql+{self.value}" + else: + raise ValueError(f"Unsupported adapter type: {self.name}") + class CloudSQLEnrichmentHandler(EnrichmentSourceHandler[beam.Row, beam.Row]): - """A handler for :class:`apache_beam.transforms.enrichment.Enrichment` - transform to interact with Google Cloud SQL databases. - - Args: - project_id (str): GCP project-id of the Cloud SQL instance. - region_id (str): GCP region-id of the Cloud SQL instance. - instance_id (str): GCP instance-id of the Cloud SQL instance. - database_type_adapter (DatabaseTypeAdapter): The type of database adapter to use. - Supported adapters are: POSTGRESQL (pg8000), MYSQL (pymysql), and SQLSERVER (pytds). - database_id (str): The id of the database to connect to. - database_user (str): The username for connecting to the database. - database_password (str): The password for connecting to the database. - table_id (str): The name of the table to query. - row_key (str): Field name from the input `beam.Row` object to use as - identifier for database querying. - row_key_fn: A lambda function that returns a string key from the - input row. Used to build/extract the identifier for the database query. - exception_level: A `enum.Enum` value from - ``apache_beam.transforms.enrichment_handlers.utils.ExceptionLevel`` - to set the level when no matching record is found from the database query. - Defaults to ``ExceptionLevel.WARN``. + """ + Enrichment handler for Cloud SQL databases. + + This handler is designed to work with the + :class:`apache_beam.transforms.enrichment.Enrichment` transform. + + To use this handler, you need to provide either of the following combinations: + * `table_id`, `where_clause_template`, `where_clause_fields` + * `table_id`, `where_clause_template`, `where_clause_value_fn` + * `query_fn` + + By default, the handler retrieves all columns from the specified table. + To limit the columns, use the `column_names` parameter to specify + the desired column names. + + This handler queries the Cloud SQL database per element by default. + To enable batching, set the `min_batch_size` and `max_batch_size` parameters. + These values control the batching behavior in the + :class:`apache_beam.transforms.utils.BatchElements` transform. + + NOTE: Batching is not supported when using the `query_fn` parameter. """ def __init__( self, - region_id: str, - project_id: str, - instance_id: str, database_type_adapter: DatabaseTypeAdapter, - database_id: str, + database_address: str, database_user: str, database_password: str, - table_id: str, - row_key: str = "", + database_id: str, *, - row_key_fn: Optional[RowKeyFn] = None, - exception_level: ExceptionLevel = ExceptionLevel.WARN, + table_id: str = "", + where_clause_template: str = "", + where_clause_fields: Optional[list[str]] = None, + where_clause_value_fn: Optional[ConditionValueFn] = None, + query_fn: Optional[QueryFn] = None, + column_names: Optional[list[str]] = None, + min_batch_size: int = 1, + max_batch_size: int = 10000, + **kwargs, ): - self._project_id = project_id - self._region_id = region_id - self._instance_id = instance_id + """ + Example Usage: + handler = CloudSQLEnrichmentHandler( + database_type_adapter=adapter, + database_address='127.0.0.1:5432', + database_user='user', + database_password='password', + database_id='my_database', + table_id='my_table', + where_clause_template="id = '{}'", + where_clause_fields=['id'], + min_batch_size=2, + max_batch_size=100 + ) + + Args: + database_type_adapter: Adapter to handle specific database type operations + (e.g., MySQL, PostgreSQL). + database_address (str): Address or hostname of the Cloud SQL database, in + the form `:`. The port is optional if the database uses + the default port. + database_user (str): Username for accessing the database. + database_password (str): Password for accessing the database. + database_id (str): Identifier for the database to query. + table_id (str): Name of the table to query in the Cloud SQL database. + where_clause_template (str): A template string for the `WHERE` clause + in the SQL query with placeholders (`{}`) for dynamic filtering + based on input data. + where_clause_fields (Optional[list[str]]): List of field names from the input + `beam.Row` used to construct the `WHERE` clause if `where_clause_value_fn` + is not provided. + where_clause_value_fn (Optional[Callable[[beam.Row], Any]]): Function that + takes a `beam.Row` and returns a list of values to populate the + placeholders `{}` in the `WHERE` clause. + query_fn (Optional[Callable[[beam.Row], str]]): Function that takes a + `beam.Row` and returns a complete SQL query string. + column_names (Optional[list[str]]): List of column names to select from the + Cloud SQL table. If not provided, all columns (`*`) are selected. + min_batch_size (int): Minimum number of rows to batch together when + querying the database. Defaults to 1 if `query_fn` is not used. + max_batch_size (int): Maximum number of rows to batch together. Defaults + to 10,000 if `query_fn` is not used. + **kwargs: Additional keyword arguments for database connection or query handling. + + Note: + * `min_batch_size` and `max_batch_size` cannot be used if `query_fn` is provided. + * Either `where_clause_fields` or `where_clause_value_fn` must be provided + for query construction if `query_fn` is not provided. + * Ensure that the database user has the necessary permissions to query the + specified table. + """ + _validate_cloudsql_metadata( + table_id, + where_clause_template, + where_clause_fields, + where_clause_value_fn, + query_fn) self._database_type_adapter = database_type_adapter self._database_id = database_id self._database_user = database_user self._database_password = database_password + self._database_address = database_address self._table_id = table_id - self._row_key = row_key - self._row_key_fn = row_key_fn - self._exception_level = exception_level - if ((not self._row_key_fn and not self._row_key) or - bool(self._row_key_fn and self._row_key)): - raise ValueError( - "Please specify exactly one of `row_key` or a lambda " - "function with `row_key_fn` to extract the row key " - "from the input row.") + self._where_clause_template = where_clause_template + self._where_clause_fields = where_clause_fields + self._where_clause_value_fn = where_clause_value_fn + self._query_fn = query_fn + self._column_names = ",".join(column_names) if column_names else "*" + self.query_template = f"SELECT {self._column_names} FROM {self._table_id} WHERE {self._where_clause_template}" + self.kwargs = kwargs + self._batching_kwargs = {} + if not query_fn: + self._batching_kwargs['min_batch_size'] = min_batch_size + self._batching_kwargs['max_batch_size'] = max_batch_size def __enter__(self): - """Connect to the the Cloud SQL instance.""" - self.connector = Connector() - self.client = self.connector.connect( - f"{self._project_id}:{self._region_id}:{self._instance_id}", - driver=self._database_type_adapter.value, - db=self._database_id, - user=self._database_user, - password=self._database_password, - ) - self.cursor = self.client.cursor() - - def __call__(self, request: beam.Row, *args, **kwargs): - """ - Executes a query to the Cloud SQL instance and returns - a `Tuple` of request and response. + db_url = self._get_db_url() + self._engine = create_engine(db_url) + self._connection = self._engine.connect() - Args: - request: the input `beam.Row` to enrich. - """ - response_dict: dict[str, Any] = {} - row_key_str: str = "" + def _get_db_url(self) -> str: + dialect = self._database_type_adapter.to_sqlalchemy_dialect() + string = f"{dialect}://{self._database_user}:{self._database_password}@{self._database_address}/{self._database_id}" + return string + def _execute_query(self, query: str, is_batch: bool, **params): try: - if self._row_key_fn: - self._row_key, row_key = self._row_key_fn(request) + result = self._connection.execute(text(query), **params) + if is_batch: + return [row._asdict() for row in result] else: - request_dict = request._asdict() - row_key_str = str(request_dict[self._row_key]) - row_key = row_key_str - - query = f"SELECT * FROM {self._table_id} WHERE {self._row_key} = %s" - self.cursor.execute(query, (row_key, )) - result = self.cursor.fetchone() - - if result: - columns = [col[0] for col in self.cursor.description] - for i, value in enumerate(result): - response_dict[columns[i]] = value - elif self._exception_level == ExceptionLevel.WARN: - _LOGGER.warning( - 'No matching record found for row_key: %s in table: %s', - row_key_str, - self._table_id) - elif self._exception_level == ExceptionLevel.RAISE: - raise ValueError( - 'No matching record found for row_key: %s in table: %s' % - (row_key_str, self._table_id)) - except KeyError: - raise KeyError('row_key %s not found in input PCollection.' % row_key_str) - except Exception as e: - raise e - - return request, beam.Row(**response_dict) + return result.first()._asdict() + except RuntimeError as e: + raise RuntimeError( + f'Could not execute the query: {query}. Please check if ' + f'the query is properly formatted and the BigQuery ' + f'table exists. {e}') + + def __call__(self, request: Union[beam.Row, list[beam.Row]], *args, **kwargs): + if isinstance(request, list): + values, responses = [], [] + requests_map: dict[Any, Any] = {} + batch_size = len(request) + raw_query = self.query_template + + # For multiple requests in the batch, combine the WHERE clause conditions + # using 'OR' and update the query template to handle all requests. + if batch_size > 1: + where_clause_template_batched = ' OR '.join( + [fr'({self._where_clause_template})'] * batch_size) + raw_query = self.query_template.replace( + self._where_clause_template, where_clause_template_batched) + + # Extract where_clause_fields values and map the generated request key to + # the original request object. + for req in request: + request_dict = req._asdict() + try: + current_values = ( + self._where_clause_value_fn(req) if self._where_clause_value_fn + else [request_dict[field] for field in self._where_clause_fields]) + except KeyError as e: + raise KeyError( + "Make sure the values passed in `where_clause_fields` are the " + "keys in the input `beam.Row`." + str(e)) + values.extend(current_values) + requests_map[self.create_row_key(req)] = req + + # Formulate the query, execute it, and return a list of original requests + # paired with their responses. + query = raw_query.format(*values) + responses_dict = self._execute_query(query, is_batch=True) + for response in responses_dict: + response_row = beam.Row(**response) + response_key = self.create_row_key(response_row) + if response_key in requests_map: + responses.append((requests_map[response_key], response_row)) + return responses + else: + request_dict = request._asdict() + if self._query_fn: + query = self._query_fn(request) + else: + try: + values = ( + self._where_clause_value_fn(request) + if self._where_clause_value_fn else + [request_dict[field] for field in self._where_clause_fields]) + except KeyError as e: + raise KeyError( + "Make sure the values passed in `where_clause_fields` are the " + "keys in the input `beam.Row`." + str(e)) + query = self.query_template.format(*values) + response_dict = self._execute_query(query, is_batch=False) + return request, beam.Row(**response_dict) + + def create_row_key(self, row: beam.Row): + if self._where_clause_value_fn: + return tuple(self._where_clause_value_fn(row)) + if self._where_clause_fields: + row_dict = row._asdict() + return ( + tuple( + row_dict[where_clause_field] + for where_clause_field in self._where_clause_fields)) + raise ValueError( + "Either where_clause_fields or where_clause_value_fn must be specified") def __exit__(self, exc_type, exc_val, exc_tb): - """Clean the instantiated Cloud SQL client.""" - self.cursor.close() - self.client.close() - self.connector.close() - self.cursor, self.client, self.connector = None, None, None - - def get_cache_key(self, request: beam.Row) -> str: - """Returns a string formatted with row key since it is unique to - a request made to the Cloud SQL instance.""" - if self._row_key_fn: - id, value = self._row_key_fn(request) - return f"{id}: {value}" - return f"{self._row_key}: {request._asdict()[self._row_key]}" + self._connection.close() + self._engine.dispose(close=True) + self._engine, self._connection = None, None + + def get_cache_key(self, request: Union[beam.Row, list[beam.Row]]): + if isinstance(request, list): + cache_keys = [] + for req in request: + req_dict = req._asdict() + try: + current_values = ( + self._where_clause_value_fn(req) if self._where_clause_value_fn + else [req_dict[field] for field in self._where_clause_fields]) + key = ";".join(["%s"] * len(current_values)) + cache_keys.extend([key % tuple(current_values)]) + except KeyError as e: + raise KeyError( + "Make sure the values passed in `where_clause_fields` are the " + "keys in the input `beam.Row`." + str(e)) + return cache_keys + else: + req_dict = request._asdict() + try: + current_values = ( + self._where_clause_value_fn(request) if self._where_clause_value_fn + else [req_dict[field] for field in self._where_clause_fields]) + key = ";".join(["%s"] * len(current_values)) + cache_key = key % tuple(current_values) + except KeyError as e: + raise KeyError( + "Make sure the values passed in `where_clause_fields` are the " + "keys in the input `beam.Row`." + str(e)) + return cache_key + + def batch_elements_kwargs(self) -> Mapping[str, Any]: + """Returns a kwargs suitable for `beam.BatchElements`.""" + return self._batching_kwargs From 9a1ed453fa9bf5f1fb2dcc7fb47bf05b21e03fae Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Fri, 11 Apr 2025 14:12:22 +0200 Subject: [PATCH 06/83] sdks/python: address claudevdm feedback (1)(test) --- .../enrichment_handlers/cloudsql_test.py | 48 ++++++++++++++----- 1 file changed, 36 insertions(+), 12 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_test.py index 4f2547b67fee..2fa74bf33233 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_test.py @@ -18,28 +18,52 @@ from parameterized import parameterized +# pylint: disable=ungrouped-imports try: from apache_beam.transforms.enrichment_handlers.cloudsql import CloudSQLEnrichmentHandler, DatabaseTypeAdapter - from apache_beam.transforms.enrichment_handlers.cloudsql_it_test import _row_key_fn + from apache_beam.transforms.enrichment_handlers.cloudsql_it_test import where_clause_value_fn + from apache_beam.transforms.enrichment_handlers.cloudsql_it_test import query_fn except ImportError: - raise unittest.SkipTest('Cloud SQL test dependencies are not installed.') + raise unittest.SkipTest('Google Cloud SQL dependencies are not installed.') -class TestCloudSQLEnrichmentHandler(unittest.TestCase): - @parameterized.expand([('product_id', _row_key_fn), ('', None)]) - def test_cloud_sql_enrichment_invalid_args(self, row_key, row_key_fn): +class TestCloudSQLEnrichment(unittest.TestCase): + @parameterized.expand([ + ("", "", [], None, None, 1, 2), + ("table", "", ["id"], where_clause_value_fn, None, 2, 10), + ("table", "id='{}'", ["id"], where_clause_value_fn, None, 2, 10), + ("table", "id='{}'", ["id"], None, query_fn, 2, 10), + ]) + def test_valid_params( + self, + table_id, + where_clause_template, + where_clause_fields, + where_clause_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 `where_clause_template` or `query_fn` is not provided. + TC 3: Both `where_clause_fields` and `where_clause_value_fn` are provided. + TC 4: Query construction details are provided along with `query_fn`. + """ with self.assertRaises(ValueError): _ = CloudSQLEnrichmentHandler( - project_id='apache-beam-testing', - region_id='us-east1', - instance_id='beam-test', - table_id='cloudsql-enrichment-test', database_type_adapter=DatabaseTypeAdapter.POSTGRESQL, - database_id='', + database_address='', database_user='', database_password='', - row_key=row_key, - row_key_fn=row_key_fn) + database_id='', + table_id=table_id, + where_clause_template=where_clause_template, + where_clause_fields=where_clause_fields, + where_clause_value_fn=where_clause_value_fn, + query_fn=query_fn, + min_batch_size=min_batch_size, + max_batch_size=max_batch_size, + ) if __name__ == '__main__': From 9261459ae1d5d43777d3fd9b9289c0ff81547200 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Fri, 11 Apr 2025 14:13:10 +0200 Subject: [PATCH 07/83] sdks/python: address claudevdm feedback (1)(itest) --- .../enrichment_handlers/cloudsql_it_test.py | 639 +++++++++--------- 1 file changed, 330 insertions(+), 309 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py index 63b9ee2d17b9..c7939b09c07d 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py @@ -14,381 +14,402 @@ # See the License for the specific language governing permissions and # limitations under the License. # +import functools import logging import unittest from unittest.mock import MagicMock + import pytest + import apache_beam as beam from apache_beam.coders import coders from apache_beam.testing.test_pipeline import TestPipeline -from apache_beam.testing.util import BeamAssertException -from apache_beam.transforms.enrichment import Enrichment -from apache_beam.transforms.enrichment_handlers.cloudsql import ( - CloudSQLEnrichmentHandler, - DatabaseTypeAdapter, - ExceptionLevel, -) -from testcontainers.redis import RedisContainer -from google.cloud.sql.connector import Connector -import os +from apache_beam.testing.util import assert_that +from apache_beam.testing.util import equal_to + +# pylint: disable=ungrouped-imports +try: + from testcontainers.postgres import PostgresContainer + from testcontainers.redis import RedisContainer + from sqlalchemy import create_engine, MetaData, Table, Column, Integer, String + from apache_beam.transforms.enrichment import Enrichment + from apache_beam.transforms.enrichment_handlers.cloudsql import ( + CloudSQLEnrichmentHandler, DatabaseTypeAdapter, DatabaseTypeAdapter) +except ImportError: + raise unittest.SkipTest('Google Cloud SQL dependencies are not installed.') _LOGGER = logging.getLogger(__name__) -def _row_key_fn(request: beam.Row, key_id="product_id") -> tuple[str]: - key_value = str(getattr(request, key_id)) - return (key_id, key_value) - - -class ValidateResponse(beam.DoFn): - """ValidateResponse validates if a PCollection of `beam.Row` - has the required fields.""" - def __init__( - self, - n_fields: int, - fields: list[str], - enriched_fields: dict[str, list[str]], - ): - self.n_fields = n_fields - self._fields = fields - self._enriched_fields = enriched_fields - - def process(self, element: beam.Row, *args, **kwargs): - element_dict = element.as_dict() - if len(element_dict.keys()) != self.n_fields: - raise BeamAssertException( - "Expected %d fields in enriched PCollection:" % self.n_fields) - - for field in self._fields: - if field not in element_dict or element_dict[field] is None: - raise BeamAssertException(f"Expected a not None field: {field}") - - for key in self._enriched_fields: - if key not in element_dict: - raise BeamAssertException( - f"Response from Cloud SQL should contain {key} column.") - - -def create_rows(cursor): - """Insert test rows into the Cloud SQL database table.""" - cursor.execute( - """ - CREATE TABLE IF NOT EXISTS products ( - product_id SERIAL PRIMARY KEY, - product_name VARCHAR(255), - product_stock INT +def where_clause_value_fn(row: beam.Row): + return [row.id] # type: ignore[attr-defined] + + +def query_fn(table, row: beam.Row): + return f"SELECT * FROM `{table}` WHERE id = {row.id}" # type: ignore[attr-defined] + + +@pytest.mark.uses_testcontainer +class CloudSQLEnrichmentIT(unittest.TestCase): + @classmethod + def setUpClass(cls): + cls._sql_client_retries = 3 + cls._start_sql_db_container() + + @classmethod + def tearDownClass(cls): + cls._stop_sql_db_container() + + @classmethod + def _start_sql_db_container(cls): + for i in range(cls._sql_client_retries): + try: + cls._sql_db_container = PostgresContainer(image="postgres:16") + cls._sql_db_container.start() + cls.sql_db_container_host = cls._sql_db_container.get_container_host_ip( ) - """) - cursor.execute( - """ - INSERT INTO products (product_name, product_stock) - VALUES - ('pixel 5', 2), - ('pixel 6', 4), - ('pixel 7', 20), - ('pixel 8', 10), - ('iphone 11', 3), - ('iphone 12', 7), - ('iphone 13', 8), - ('iphone 14', 3) - ON CONFLICT DO NOTHING - """) + cls.sql_db_container_port = cls._sql_db_container.get_exposed_port(5432) + cls.database_type_adapter = DatabaseTypeAdapter.POSTGRESQL + cls.sql_db_user, cls.sql_db_password, cls.sql_db_id = "test", "test", "test" + _LOGGER.info( + f"PostgreSQL container started successfully on {cls.get_db_address()}." + ) + break + except Exception as e: + _LOGGER.warning( + f"Retry {i + 1}/{cls._sql_client_retries}: Failed to start PostgreSQL container. Reason: {e}" + ) + if i == cls._sql_client_retries - 1: + _LOGGER.error( + f"Unable to start PostgreSQL container for IO tests after {cls._sql_client_retries} retries. Tests cannot proceed." + ) + raise e + + @classmethod + def _stop_sql_db_container(cls): + try: + _LOGGER.info("Stopping PostgreSQL container.") + cls._sql_db_container.stop() + cls._sql_db_container = None + _LOGGER.info("PostgreSQL container stopped successfully.") + except Exception as e: + _LOGGER.warning( + f"Error encountered while stopping PostgreSQL container: {e}") + + @classmethod + def get_db_address(cls): + return f"{cls.sql_db_container_host}:{cls.sql_db_container_port}" @pytest.mark.uses_testcontainer -class TestCloudSQLEnrichment(unittest.TestCase): +class TestCloudSQLEnrichment(CloudSQLEnrichmentIT): + _table_id = "product_details" + _table_data = [ + { + "id": 1, "name": "A", 'quantity': 2, 'distribution_center_id': 3 + }, + { + "id": 2, "name": "B", 'quantity': 3, 'distribution_center_id': 1 + }, + { + "id": 3, "name": "C", 'quantity': 10, 'distribution_center_id': 4 + }, + { + "id": 4, "name": "D", 'quantity': 1, 'distribution_center_id': 3 + }, + { + "id": 5, "name": "C", 'quantity': 100, 'distribution_center_id': 4 + }, + { + "id": 6, "name": "D", 'quantity': 11, 'distribution_center_id': 3 + }, + { + "id": 7, "name": "C", 'quantity': 7, 'distribution_center_id': 1 + }, + { + "id": 8, "name": "D", 'quantity': 4, 'distribution_center_id': 1 + }, + ] + @classmethod def setUpClass(cls): - cls.project_id = "apache-beam-testing" - cls.region_id = "us-central1" - cls.instance_id = "beam-test" - cls.database_id = "postgres" - cls.database_user = os.getenv("BEAM_TEST_CLOUDSQL_PG_USER") - cls.database_password = os.getenv("BEAM_TEST_CLOUDSQL_PG_PASSWORD") - cls.table_id = "products" - cls.row_key = "product_id" - cls.database_type_adapter = DatabaseTypeAdapter.POSTGRESQL - cls.req = [ - beam.Row(sale_id=1, customer_id=1, product_id=1, quantity=1), - beam.Row(sale_id=3, customer_id=3, product_id=2, quantity=3), - beam.Row(sale_id=5, customer_id=5, product_id=3, quantity=2), - beam.Row(sale_id=7, customer_id=7, product_id=4, quantity=1), - ] - cls.connector = Connector() - cls.client = cls.connector.connect( - f"{cls.project_id}:{cls.region_id}:{cls.instance_id}", - driver=cls.database_type_adapter.value, - db=cls.database_id, - user=cls.database_user, - password=cls.database_password, + super(TestCloudSQLEnrichment, cls).setUpClass() + cls.create_table(cls._table_id) + cls._cache_client_retries = 3 + + @classmethod + def create_table(cls, table_id): + cls._engine = create_engine(cls._get_db_url()) + + # Define the table schema. + metadata = MetaData() + table = Table( + table_id, + metadata, + Column("id", Integer, primary_key=True), + Column("name", String, nullable=False), + Column("quantity", Integer, nullable=False), + Column("distribution_center_id", Integer, nullable=False), ) - cls.cursor = cls.client.cursor() - create_rows(cls.cursor) - cls.cache_client_retries = 3 + + # Create the table in the database. + metadata.create_all(cls._engine) + + # Insert data into the table. + with cls._engine.connect() as connection: + transaction = connection.begin() + try: + connection.execute(table.insert(), cls._table_data) + transaction.commit() + except Exception as e: + transaction.rollback() + raise e + + @classmethod + def _get_db_url(cls): + dialect = cls.database_type_adapter.to_sqlalchemy_dialect() + db_url = f"{dialect}://{cls.sql_db_user}:{cls.sql_db_password}@{cls.get_db_address()}/{cls.sql_db_id}" + return db_url + + @pytest.fixture + def cache_container(self): + self._start_cache_container() + + # Hand control to the test. + yield + + self._cache_container.stop() + self._cache_container = None def _start_cache_container(self): - for i in range(self.cache_client_retries): + for i in range(self._cache_client_retries): try: - self.container = RedisContainer(image="redis:7.2.4") - self.container.start() - self.host = self.container.get_container_host_ip() - self.port = self.container.get_exposed_port(6379) - self.cache_client = self.container.get_client() + self._cache_container = RedisContainer(image="redis:7.2.4") + self._cache_container.start() + self._cache_container_host = self._cache_container.get_container_host_ip( + ) + self._cache_container_port = self._cache_container.get_exposed_port( + 6379) + self._cache_client = self._cache_container.get_client() break except Exception as e: - if i == self.cache_client_retries - 1: + if i == self._cache_client_retries - 1: _LOGGER.error( - f"Unable to start redis container for RRIO tests after {self.cache_client_retries} retries." + f"Unable to start redis container for RRIO tests after {self._cache_client_retries} retries." ) raise e @classmethod def tearDownClass(cls): - cls.cursor.close() - cls.client.close() - cls.connector.close() - cls.cursor, cls.client, cls.connector = None, None, None - - def test_enrichment_with_cloudsql(self): - expected_fields = [ - "sale_id", - "customer_id", - "product_id", - "quantity", - "product_name", - "product_stock", + cls._engine.dispose(close=True) + super(TestCloudSQLEnrichment, cls).tearDownClass() + cls._engine = None + + def test_cloudsql_enrichment(self): + expected_rows = [ + beam.Row(id=1, name="A", quantity=2, distribution_center_id=3), + beam.Row(id=2, name="B", quantity=3, distribution_center_id=1) ] - expected_enriched_fields = ["product_id", "product_name", "product_stock"] - cloudsql = CloudSQLEnrichmentHandler( - region_id=self.region_id, - project_id=self.project_id, - instance_id=self.instance_id, + fields = ['id'] + requests = [ + beam.Row(id=1, name='A'), + beam.Row(id=2, name='B'), + ] + handler = CloudSQLEnrichmentHandler( database_type_adapter=self.database_type_adapter, - database_id=self.database_id, - database_user=self.database_user, - database_password=self.database_password, - table_id=self.table_id, - row_key=self.row_key, + database_address=self.get_db_address(), + database_user=self.sql_db_user, + database_password=self.sql_db_password, + database_id=self.sql_db_id, + table_id=self._table_id, + where_clause_template="id = {}", + where_clause_fields=fields, + min_batch_size=1, + max_batch_size=100, ) with TestPipeline(is_integration_test=True) as test_pipeline: - _ = ( - test_pipeline - | "Create" >> beam.Create(self.req) - | "Enrich W/ CloudSQL" >> Enrichment(cloudsql) - | "Validate Response" >> beam.ParDo( - ValidateResponse( - len(expected_fields), - expected_fields, - expected_enriched_fields, - ))) - - def test_enrichment_with_cloudsql_no_enrichment(self): - expected_fields = ["sale_id", "customer_id", "product_id", "quantity"] - expected_enriched_fields = {} - cloudsql = CloudSQLEnrichmentHandler( - region_id=self.region_id, - project_id=self.project_id, - instance_id=self.instance_id, + pcoll = (test_pipeline | beam.Create(requests) | Enrichment(handler)) + + assert_that(pcoll, equal_to(expected_rows)) + + def test_cloudsql_enrichment_batched(self): + expected_rows = [ + beam.Row(id=1, name="A", quantity=2, distribution_center_id=3), + beam.Row(id=2, name="B", quantity=3, distribution_center_id=1) + ] + fields = ['id'] + requests = [ + beam.Row(id=1, name='A'), + beam.Row(id=2, name='B'), + ] + handler = CloudSQLEnrichmentHandler( database_type_adapter=self.database_type_adapter, - database_id=self.database_id, - database_user=self.database_user, - database_password=self.database_password, - table_id=self.table_id, - row_key=self.row_key, + database_address=self.get_db_address(), + database_user=self.sql_db_user, + database_password=self.sql_db_password, + database_id=self.sql_db_id, + table_id=self._table_id, + where_clause_template="id = {}", + where_clause_fields=fields, + min_batch_size=2, + max_batch_size=100, ) - req = [beam.Row(sale_id=1, customer_id=1, product_id=99, quantity=1)] with TestPipeline(is_integration_test=True) as test_pipeline: - _ = ( - test_pipeline - | "Create" >> beam.Create(req) - | "Enrich W/ CloudSQL" >> Enrichment(cloudsql) - | "Validate Response" >> beam.ParDo( - ValidateResponse( - len(expected_fields), - expected_fields, - expected_enriched_fields, - ))) - - def test_enrichment_with_cloudsql_raises_key_error(self): - cloudsql = CloudSQLEnrichmentHandler( - region_id=self.region_id, - project_id=self.project_id, - instance_id=self.instance_id, + pcoll = (test_pipeline | beam.Create(requests) | Enrichment(handler)) + + assert_that(pcoll, equal_to(expected_rows)) + + def test_cloudsql_enrichment_batched_multiple_fields(self): + expected_rows = [ + beam.Row(id=1, distribution_center_id=3, name="A", quantity=2), + beam.Row(id=2, distribution_center_id=1, name="B", quantity=3) + ] + fields = ['id', 'distribution_center_id'] + requests = [ + beam.Row(id=1, distribution_center_id=3), + beam.Row(id=2, distribution_center_id=1), + ] + handler = CloudSQLEnrichmentHandler( database_type_adapter=self.database_type_adapter, - database_id=self.database_id, - database_user=self.database_user, - database_password=self.database_password, - table_id=self.table_id, - row_key="car_name", + database_address=self.get_db_address(), + database_user=self.sql_db_user, + database_password=self.sql_db_password, + database_id=self.sql_db_id, + table_id=self._table_id, + where_clause_template="id = {} AND distribution_center_id = {}", + where_clause_fields=fields, + min_batch_size=8, + max_batch_size=100, ) - with self.assertRaises(KeyError): - test_pipeline = TestPipeline() - _ = ( - test_pipeline - | "Create" >> beam.Create(self.req) - | "Enrich W/ CloudSQL" >> Enrichment(cloudsql)) - res = test_pipeline.run() - res.wait_until_finish() + with TestPipeline(is_integration_test=True) as test_pipeline: + pcoll = (test_pipeline | beam.Create(requests) | Enrichment(handler)) + + assert_that(pcoll, equal_to(expected_rows)) - def test_enrichment_with_cloudsql_raises_not_found(self): - """Raises a database error when the GCP Cloud SQL table doesn't exist.""" - table_id = "invalid_table" - cloudsql = CloudSQLEnrichmentHandler( - region_id=self.region_id, - project_id=self.project_id, - instance_id=self.instance_id, + def test_cloudsql_enrichment_with_query_fn(self): + expected_rows = [ + beam.Row(id=1, name="A", quantity=2, distribution_center_id=3), + beam.Row(id=2, name="B", quantity=3, distribution_center_id=1) + ] + requests = [ + beam.Row(id=1, name='A'), + beam.Row(id=2, name='B'), + ] + fn = functools.partial(query_fn, self._table_id) + handler = CloudSQLEnrichmentHandler( database_type_adapter=self.database_type_adapter, - database_id=self.database_id, - database_user=self.database_user, - database_password=self.database_password, - table_id=table_id, - row_key=self.row_key, - ) - try: - test_pipeline = beam.Pipeline() - _ = ( - test_pipeline - | "Create" >> beam.Create(self.req) - | "Enrich W/ CloudSQL" >> Enrichment(cloudsql)) - res = test_pipeline.run() - res.wait_until_finish() - except (PgDatabaseError, RuntimeError) as e: - self.assertIn(f'relation "{table_id}" does not exist', str(e)) - - def test_enrichment_with_cloudsql_exception_level(self): - """raises a `ValueError` exception when the GCP Cloud SQL query returns - an empty row.""" - cloudsql = CloudSQLEnrichmentHandler( - region_id=self.region_id, - project_id=self.project_id, - instance_id=self.instance_id, + database_address=self.get_db_address(), + database_user=self.sql_db_user, + database_password=self.sql_db_password, + database_id=self.sql_db_id, + query_fn=fn) + with TestPipeline(is_integration_test=True) as test_pipeline: + pcoll = (test_pipeline | beam.Create(requests) | Enrichment(handler)) + + assert_that(pcoll, equal_to(expected_rows)) + + def test_cloudsql_enrichment_with_condition_value_fn(self): + expected_rows = [ + beam.Row(id=1, name="A", quantity=2, distribution_center_id=3), + beam.Row(id=2, name="B", quantity=3, distribution_center_id=1) + ] + requests = [ + beam.Row(id=1, name='A'), + beam.Row(id=2, name='B'), + ] + handler = CloudSQLEnrichmentHandler( database_type_adapter=self.database_type_adapter, - database_id=self.database_id, - database_user=self.database_user, - database_password=self.database_password, - table_id=self.table_id, - row_key=self.row_key, - exception_level=ExceptionLevel.RAISE, - ) - req = [beam.Row(sale_id=1, customer_id=1, product_id=11, quantity=1)] - with self.assertRaises(ValueError): - test_pipeline = beam.Pipeline() - _ = ( - test_pipeline - | "Create" >> beam.Create(req) - | "Enrich W/ CloudSQL" >> Enrichment(cloudsql)) - res = test_pipeline.run() - res.wait_until_finish() + database_address=self.get_db_address(), + database_user=self.sql_db_user, + database_password=self.sql_db_password, + database_id=self.sql_db_id, + table_id=self._table_id, + where_clause_template="id = {}", + where_clause_value_fn=where_clause_value_fn, + min_batch_size=2, + max_batch_size=100) + with TestPipeline(is_integration_test=True) as test_pipeline: + pcoll = (test_pipeline | beam.Create(requests) | Enrichment(handler)) - def test_cloudsql_enrichment_with_lambda(self): - expected_fields = [ - "sale_id", - "customer_id", - "product_id", - "quantity", - "product_name", - "product_stock", + assert_that(pcoll, equal_to(expected_rows)) + + def test_cloudsql_enrichment_table_nonexistent_runtime_error_raised(self): + requests = [ + beam.Row(id=1, name='A'), + beam.Row(id=2, name='B'), ] - expected_enriched_fields = ["product_id", "product_name", "product_stock"] - cloudsql = CloudSQLEnrichmentHandler( - region_id=self.region_id, - project_id=self.project_id, - instance_id=self.instance_id, + handler = CloudSQLEnrichmentHandler( database_type_adapter=self.database_type_adapter, - database_id=self.database_id, - database_user=self.database_user, - database_password=self.database_password, - table_id=self.table_id, - row_key_fn=_row_key_fn, + database_address=self.get_db_address(), + database_user=self.sql_db_user, + database_password=self.sql_db_password, + database_id=self.sql_db_id, + table_id=self._table_id, + where_clause_template="id = {}", + where_clause_value_fn=where_clause_value_fn, + column_names=["wrong_column"], ) - with TestPipeline(is_integration_test=True) as test_pipeline: + with self.assertRaises(RuntimeError): + test_pipeline = beam.Pipeline() _ = ( test_pipeline - | "Create" >> beam.Create(self.req) - | "Enrich W/ CloudSQL" >> Enrichment(cloudsql) - | "Validate Response" >> beam.ParDo( - ValidateResponse( - len(expected_fields), - expected_fields, - expected_enriched_fields))) - - @pytest.fixture - def cache_container(self): - # Setup phase: start the container. - self._start_cache_container() - - # Hand control to the test. - yield - - # Cleanup phase: stop the container. It runs after the test completion - # even if it failed. - self.container.stop() - self.container = None + | "Create" >> beam.Create(requests) + | "Enrichment" >> Enrichment(handler)) + res = test_pipeline.run() + res.wait_until_finish() @pytest.mark.usefixtures("cache_container") def test_cloudsql_enrichment_with_redis(self): - expected_fields = [ - "sale_id", - "customer_id", - "product_id", - "quantity", - "product_name", - "product_stock", + requests = [ + beam.Row(id=1, name='A'), + beam.Row(id=2, name='B'), + ] + expected_rows = [ + beam.Row(id=1, name="A", quantity=2, distribution_center_id=3), + beam.Row(id=2, name="B", quantity=3, distribution_center_id=1) ] - expected_enriched_fields = ["product_id", "product_name", "product_stock"] - cloudsql = CloudSQLEnrichmentHandler( - region_id=self.region_id, - project_id=self.project_id, - instance_id=self.instance_id, + handler = CloudSQLEnrichmentHandler( database_type_adapter=self.database_type_adapter, - database_id=self.database_id, - database_user=self.database_user, - database_password=self.database_password, - table_id=self.table_id, - row_key_fn=_row_key_fn, - ) + database_address=self.get_db_address(), + database_user=self.sql_db_user, + database_password=self.sql_db_password, + database_id=self.sql_db_id, + table_id=self._table_id, + where_clause_template="id = {}", + where_clause_value_fn=where_clause_value_fn, + min_batch_size=2, + max_batch_size=100) with TestPipeline(is_integration_test=True) as test_pipeline: - _ = ( + pcoll_populate_cache = ( test_pipeline - | "Create1" >> beam.Create(self.req) - | "Enrich W/ CloudSQL1" >> Enrichment(cloudsql).with_redis_cache( - self.host, self.port, 300) - | "Validate Response" >> beam.ParDo( - ValidateResponse( - len(expected_fields), - expected_fields, - expected_enriched_fields, - ))) + | beam.Create(requests) + | Enrichment(handler).with_redis_cache(self.host, self.port)) + + assert_that(pcoll_populate_cache, equal_to(expected_rows)) # Manually check cache entry to verify entries were correctly stored. c = coders.StrUtf8Coder() - for req in self.req: - key = cloudsql.get_cache_key(req) - response = self.cache_client.get(c.encode(key)) + for req in requests: + key = handler.get_cache_key(req) + response = self._cache_client.get(c.encode(key)) if not response: raise ValueError("No cache entry found for %s" % key) - # Mock the CloudSQL handler to avoid actual database calls. + # Mock the CloudSQL enrichment handler to avoid actual database calls. # This simulates a cache hit scenario by returning predefined data. actual = CloudSQLEnrichmentHandler.__call__ - CloudSQLEnrichmentHandler.__call__ = MagicMock( - return_value=( - beam.Row(sale_id=1, customer_id=1, product_id=1, quantity=1), - beam.Row(), - )) + CloudSQLEnrichmentHandler.__call__ = MagicMock(return_value=(beam.Row())) # Run a second pipeline to verify cache is being used. with TestPipeline(is_integration_test=True) as test_pipeline: - _ = ( + pcoll_cached = ( test_pipeline - | "Create2" >> beam.Create(self.req) - | "Enrich W/ CloudSQL2" >> Enrichment(cloudsql).with_redis_cache( - self.host, self.port) - | "Validate Response" >> beam.ParDo( - ValidateResponse( - len(expected_fields), - expected_fields, - expected_enriched_fields))) + | beam.Create(requests) + | Enrichment(handler).with_redis_cache(self.host, self.port)) + + assert_that(pcoll_cached, equal_to(expected_rows)) + + # Restore the original CloudSQL enrichment handler implementation. CloudSQLEnrichmentHandler.__call__ = actual From ecbb61da2a5762b14bf19212f749e15b63012709 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Fri, 11 Apr 2025 14:27:47 +0200 Subject: [PATCH 08/83] sdks: update doc `CloudSQLEnrichmentHandler` --- .../transforms/elementwise/enrichment.py | 32 +++++++++---------- .../transforms/elementwise/enrichment_test.py | 2 ++ 2 files changed, 18 insertions(+), 16 deletions(-) diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py index c88696bbc24d..41b22863e81c 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py @@ -50,6 +50,7 @@ def enrichment_with_bigtable(): | "Print" >> beam.Map(print)) # [END enrichment_with_bigtable] + def enrichment_with_cloudsql(): # [START enrichment_with_cloudsql] import apache_beam as beam @@ -57,14 +58,14 @@ def enrichment_with_cloudsql(): from apache_beam.transforms.enrichment_handlers.cloudsql import CloudSQLEnrichmentHandler, DatabaseTypeAdapter import os - project_id = 'apache-beam-testing' - region_id = 'us-east1' - instance_id = 'beam-test' - table_id = 'cloudsql-enrichment-test' - database_id = 'test-database' - database_user = os.getenv("BEAM_TEST_CLOUDSQL_PG_USER") - database_password = os.getenv("BEAM_TEST_CLOUDSQL_PG_PASSWORD") - row_key = 'product_id' + database_type_adapter = DatabaseTypeAdapter.POSTGRESQL + database_address = "10.0.0.42:5432" + database_user = "test" + database_password = os.getenv("DB_PASSWORD") + database_id = "test" + table_id = "products" + where_clause_template = "product_id = {}" + where_clause_fields = ["id"] data = [ beam.Row(sale_id=1, customer_id=1, product_id=1, quantity=1), @@ -73,16 +74,14 @@ def enrichment_with_cloudsql(): ] cloudsql_handler = CloudSQLEnrichmentHandler( - project_id=project_id, - region_id=region_id, - instance_id=instance_id, - table_id=table_id, - database_type_adapter=DatabaseTypeAdapter.POSTGRESQL, - database_id=database_id, + database_type_adapter=database_type_adapter, + database_address=database_address, database_user=database_user, database_password=database_password, - row_key=row_key - ) + database_id=database_id, + table_id=table_id, + where_clause_template=where_clause_template, + where_clause_fields=where_clause_fields) with beam.Pipeline() as p: _ = ( p @@ -91,6 +90,7 @@ def enrichment_with_cloudsql(): | "Print" >> beam.Map(print)) # [END enrichment_with_cloudsql] + def enrichment_with_vertex_ai(): # [START enrichment_with_vertex_ai] import apache_beam as beam diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py index 7086261c254e..e217bb3b49da 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py @@ -41,6 +41,7 @@ def validate_enrichment_with_bigtable(): [END enrichment_with_bigtable]'''.splitlines()[1:-1] return expected + def validate_enrichment_with_cloudsql(): expected = '''[START enrichment_with_cloudsql] Row(sale_id=1, customer_id=1, product_id=1, quantity=1, product={'product_id': '1', 'product_name': 'pixel 5', 'product_stock': '2'}) @@ -49,6 +50,7 @@ def validate_enrichment_with_cloudsql(): [END enrichment_with_cloudsql]'''.splitlines()[1:-1] return expected + def validate_enrichment_with_vertex_ai(): expected = '''[START enrichment_with_vertex_ai] Row(user_id='2963', product_id=14235, sale_price=15.0, age=12.0, state='1', gender='1', country='1') From 83e715405bed89b42aacbfdf269de4fcb52c9703 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Fri, 11 Apr 2025 20:05:54 +0200 Subject: [PATCH 09/83] sdks/python: fix linting issues --- .../transforms/elementwise/enrichment.py | 23 +- .../transforms/elementwise/enrichment_test.py | 81 +++++- .../enrichment_handlers/cloudsql.py | 37 ++- .../enrichment_handlers/cloudsql_it_test.py | 263 ++++++++++-------- .../enrichment_handlers/cloudsql_test.py | 9 +- 5 files changed, 255 insertions(+), 158 deletions(-) diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py index 41b22863e81c..13a0ad2e226f 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py @@ -55,22 +55,23 @@ def enrichment_with_cloudsql(): # [START enrichment_with_cloudsql] import apache_beam as beam from apache_beam.transforms.enrichment import Enrichment - from apache_beam.transforms.enrichment_handlers.cloudsql import CloudSQLEnrichmentHandler, DatabaseTypeAdapter + from apache_beam.transforms.enrichment_handlers.cloudsql import ( + CloudSQLEnrichmentHandler, DatabaseTypeAdapter) import os - database_type_adapter = DatabaseTypeAdapter.POSTGRESQL - database_address = "10.0.0.42:5432" - database_user = "test" - database_password = os.getenv("DB_PASSWORD") - database_id = "test" - table_id = "products" + database_type_adapter = DatabaseTypeAdapter[os.environ.get("SQL_DB_TYPE")] + database_address = os.environ.get("SQL_DB_ADDRESS") + database_user = os.environ.get("SQL_DB_USER") + database_password = os.environ.get("SQL_DB_PASSWORD") + database_id = os.environ.get("SQL_DB_ID") + table_id = os.environ.get("SQL_TABLE_ID") where_clause_template = "product_id = {}" - where_clause_fields = ["id"] + where_clause_fields = ["product_id"] data = [ - beam.Row(sale_id=1, customer_id=1, product_id=1, quantity=1), - beam.Row(sale_id=3, customer_id=3, product_id=2, quantity=3), - beam.Row(sale_id=5, customer_id=5, product_id=4, quantity=2), + beam.Row(product_id=1, name='A'), + beam.Row(product_id=2, name='B'), + beam.Row(product_id=3, name='C'), ] cloudsql_handler = CloudSQLEnrichmentHandler( diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py index e217bb3b49da..82d9caaf3a6b 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py @@ -18,16 +18,22 @@ # pytype: skip-file # pylint: disable=line-too-long +import os import unittest from io import StringIO import mock +import pytest # pylint: disable=unused-import try: - from apache_beam.examples.snippets.transforms.elementwise.enrichment import enrichment_with_bigtable, \ - enrichment_with_vertex_ai_legacy - from apache_beam.examples.snippets.transforms.elementwise.enrichment import enrichment_with_vertex_ai + from sqlalchemy import Column, Integer, String, Engine + from apache_beam.examples.snippets.transforms.elementwise.enrichment import ( + enrichment_with_bigtable, enrichment_with_vertex_ai_legacy) + from apache_beam.examples.snippets.transforms.elementwise.enrichment import ( + enrichment_with_vertex_ai, enrichment_with_cloudsql) + from apache_beam.transforms.enrichment_handlers.cloudsql_it_test import ( + CloudSQLEnrichmentTestHelper, SQLDBContainerInfo) from apache_beam.io.requestresponse import RequestResponseIO except ImportError: raise unittest.SkipTest('RequestResponseIO dependencies are not installed') @@ -44,9 +50,9 @@ def validate_enrichment_with_bigtable(): def validate_enrichment_with_cloudsql(): expected = '''[START enrichment_with_cloudsql] -Row(sale_id=1, customer_id=1, product_id=1, quantity=1, product={'product_id': '1', 'product_name': 'pixel 5', 'product_stock': '2'}) -Row(sale_id=3, customer_id=3, product_id=2, quantity=3, product={'product_id': '2', 'product_name': 'pixel 6', 'product_stock': '4'}) -Row(sale_id=5, customer_id=5, product_id=4, quantity=2, product={'product_id': '4', 'product_name': 'pixel 8', 'product_stock': '10'}) +Row(product_id=1, name='A', quantity=2, region_id=3) +Row(product_id=2, name='B', quantity=3, region_id=1) +Row(product_id=3, name='C', quantity=10, region_id=4) [END enrichment_with_cloudsql]'''.splitlines()[1:-1] return expected @@ -70,6 +76,7 @@ def validate_enrichment_with_vertex_ai_legacy(): @mock.patch('sys.stdout', new_callable=StringIO) +@pytest.mark.uses_testcontainer class EnrichmentTest(unittest.TestCase): def test_enrichment_with_bigtable(self, mock_stdout): enrichment_with_bigtable() @@ -78,11 +85,18 @@ def test_enrichment_with_bigtable(self, mock_stdout): self.assertEqual(output, expected) def test_enrichment_with_cloudsql(self, mock_stdout): - from apache_beam.examples.snippets.transforms.elementwise.enrichment import enrichment_with_cloudsql - enrichment_with_cloudsql() - output = mock_stdout.getvalue().splitlines() - expected = validate_enrichment_with_cloudsql() - self.assertEqual(output, expected) + db, engine = None, None + try: + db, engine = self.pre_cloudsql_enrichment_test() + enrichment_with_cloudsql() + output = mock_stdout.getvalue().splitlines() + expected = validate_enrichment_with_cloudsql() + self.assertEqual(output, expected) + except Exception as e: + self.fail(f"Test failed with unexpected error: {e}") + finally: + if db and engine: + self.post_cloudsql_enrichment_test(db, engine) def test_enrichment_with_vertex_ai(self, mock_stdout): enrichment_with_vertex_ai() @@ -99,6 +113,51 @@ def test_enrichment_with_vertex_ai_legacy(self, mock_stdout): self.maxDiff = None self.assertEqual(output, expected) + def pre_cloudsql_enrichment_test(self): + columns = [ + Column("product_id", Integer, primary_key=True), + Column("name", String, nullable=False), + Column("quantity", Integer, nullable=False), + Column("region_id", Integer, nullable=False), + ] + table_data = [ + { + "product_id": 1, "name": "A", 'quantity': 2, 'region_id': 3 + }, + { + "product_id": 2, "name": "B", 'quantity': 3, 'region_id': 1 + }, + { + "product_id": 3, "name": "C", 'quantity': 10, 'region_id': 4 + }, + ] + db = CloudSQLEnrichmentTestHelper.start_sql_db_container() + os.environ['SQL_DB_TYPE'] = db.adapter.name + os.environ['SQL_DB_ADDRESS'] = db.address + os.environ['SQL_DB_USER'] = db.user + os.environ['SQL_DB_PASSWORD'] = db.password + os.environ['SQL_DB_ID'] = db.id + os.environ['SQL_DB_URL'] = db.url + os.environ['SQL_TABLE_ID'] = "products" + engine = CloudSQLEnrichmentTestHelper.create_table( + table_id=os.environ.get("SQL_TABLE_ID"), + db_url=os.environ.get("SQL_DB_URL"), + columns=columns, + table_data=table_data) + return db, engine + + def post_cloudsql_enrichment_test( + self, db: SQLDBContainerInfo, engine: Engine): + engine.dispose(close=True) + CloudSQLEnrichmentTestHelper.stop_sql_db_container(db.container) + os.environ.pop('SQL_DB_TYPE', None) + os.environ.pop('SQL_DB_ADDRESS', None) + os.environ.pop('SQL_DB_USER', None) + os.environ.pop('SQL_DB_PASSWORD', None) + os.environ.pop('SQL_DB_ID', None) + os.environ.pop('SQL_DB_URL', None) + os.environ.pop('SQL_TABLE_ID', None) + if __name__ == '__main__': unittest.main() diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py index 77ab9889c8da..4f2ccea3575c 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py @@ -21,11 +21,11 @@ from typing import Optional from typing import Union -from sqlalchemy import create_engine, text +from sqlalchemy import create_engine +from sqlalchemy import text import apache_beam as beam from apache_beam.transforms.enrichment import EnrichmentSourceHandler -from apache_beam.transforms.enrichment_handlers.utils import ExceptionLevel QueryFn = Callable[[beam.Row], str] ConditionValueFn = Callable[[beam.Row], list[Any]] @@ -44,8 +44,8 @@ def _validate_cloudsql_metadata( where_clause_value_fn]): raise ValueError( "Please provide either `query_fn` or the parameters `table_id`, " - "`where_clause_template`, and `where_clause_fields/where_clause_value_fn` " - "together.") + "`where_clause_template`, and " + "`where_clause_fields/where_clause_value_fn` together.") else: if not (table_id and where_clause_template): raise ValueError( @@ -147,24 +147,25 @@ def __init__( where_clause_template (str): A template string for the `WHERE` clause in the SQL query with placeholders (`{}`) for dynamic filtering based on input data. - where_clause_fields (Optional[list[str]]): List of field names from the input - `beam.Row` used to construct the `WHERE` clause if `where_clause_value_fn` - is not provided. + where_clause_fields (Optional[list[str]]): List of field names from the + input `beam.Row` used to construct the `WHERE` clause if + `where_clause_value_fn` is not provided. where_clause_value_fn (Optional[Callable[[beam.Row], Any]]): Function that takes a `beam.Row` and returns a list of values to populate the placeholders `{}` in the `WHERE` clause. query_fn (Optional[Callable[[beam.Row], str]]): Function that takes a `beam.Row` and returns a complete SQL query string. - column_names (Optional[list[str]]): List of column names to select from the - Cloud SQL table. If not provided, all columns (`*`) are selected. + column_names (Optional[list[str]]): List of column names to select from + the Cloud SQL table. If not provided, all columns (`*`) are selected. min_batch_size (int): Minimum number of rows to batch together when querying the database. Defaults to 1 if `query_fn` is not used. max_batch_size (int): Maximum number of rows to batch together. Defaults to 10,000 if `query_fn` is not used. - **kwargs: Additional keyword arguments for database connection or query handling. + **kwargs: Additional keyword arguments for database connection or query + handling. Note: - * `min_batch_size` and `max_batch_size` cannot be used if `query_fn` is provided. + * Cannot use `min_batch_size` or `max_batch_size` with `query_fn`. * Either `where_clause_fields` or `where_clause_value_fn` must be provided for query construction if `query_fn` is not provided. * Ensure that the database user has the necessary permissions to query the @@ -183,11 +184,15 @@ def __init__( self._database_address = database_address self._table_id = table_id self._where_clause_template = where_clause_template - self._where_clause_fields = where_clause_fields self._where_clause_value_fn = where_clause_value_fn self._query_fn = query_fn + fields = where_clause_fields if where_clause_fields else [] + self._where_clause_fields = fields self._column_names = ",".join(column_names) if column_names else "*" - self.query_template = f"SELECT {self._column_names} FROM {self._table_id} WHERE {self._where_clause_template}" + self.query_template = ( + f"SELECT {self._column_names} " + f"FROM {self._table_id} " + f"WHERE {self._where_clause_template}") self.kwargs = kwargs self._batching_kwargs = {} if not query_fn: @@ -201,8 +206,10 @@ def __enter__(self): def _get_db_url(self) -> str: dialect = self._database_type_adapter.to_sqlalchemy_dialect() - string = f"{dialect}://{self._database_user}:{self._database_password}@{self._database_address}/{self._database_id}" - return string + url = ( + f"{dialect}://{self._database_user}:{self._database_password}" + f"@{self._database_address}/{self._database_id}") + return url def _execute_query(self, query: str, is_batch: bool, **params): try: diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py index c7939b09c07d..58ce8dce5263 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py @@ -17,6 +17,8 @@ import functools import logging import unittest +from dataclasses import dataclass +from typing import Optional from unittest.mock import MagicMock import pytest @@ -29,12 +31,16 @@ # pylint: disable=ungrouped-imports try: + from testcontainers.core.generic import DbContainer from testcontainers.postgres import PostgresContainer from testcontainers.redis import RedisContainer - from sqlalchemy import create_engine, MetaData, Table, Column, Integer, String + from sqlalchemy import ( + create_engine, MetaData, Table, Column, Integer, String, Engine) from apache_beam.transforms.enrichment import Enrichment from apache_beam.transforms.enrichment_handlers.cloudsql import ( - CloudSQLEnrichmentHandler, DatabaseTypeAdapter, DatabaseTypeAdapter) + CloudSQLEnrichmentHandler, + DatabaseTypeAdapter, + ) except ImportError: raise unittest.SkipTest('Google Cloud SQL dependencies are not installed.') @@ -49,61 +55,114 @@ def query_fn(table, row: beam.Row): return f"SELECT * FROM `{table}` WHERE id = {row.id}" # type: ignore[attr-defined] -@pytest.mark.uses_testcontainer -class CloudSQLEnrichmentIT(unittest.TestCase): - @classmethod - def setUpClass(cls): - cls._sql_client_retries = 3 - cls._start_sql_db_container() - - @classmethod - def tearDownClass(cls): - cls._stop_sql_db_container() - - @classmethod - def _start_sql_db_container(cls): - for i in range(cls._sql_client_retries): +@dataclass +class SQLDBContainerInfo: + adapter: DatabaseTypeAdapter + container: DbContainer + host: str + port: int + user: str + password: str + id: str + + @property + def address(self) -> str: + return f"{self.host}:{self.port}" + + @property + def url(self) -> str: + dialect = self.adapter.to_sqlalchemy_dialect() + return f"{dialect}://{self.user}:{self.password}@{self.address}/{self.id}" + + +class CloudSQLEnrichmentTestHelper: + @staticmethod + def start_sql_db_container( + sql_client_retries=3) -> Optional[SQLDBContainerInfo]: + info = None + for i in range(sql_client_retries): try: - cls._sql_db_container = PostgresContainer(image="postgres:16") - cls._sql_db_container.start() - cls.sql_db_container_host = cls._sql_db_container.get_container_host_ip( - ) - cls.sql_db_container_port = cls._sql_db_container.get_exposed_port(5432) - cls.database_type_adapter = DatabaseTypeAdapter.POSTGRESQL - cls.sql_db_user, cls.sql_db_password, cls.sql_db_id = "test", "test", "test" + database_type_adapter = DatabaseTypeAdapter.POSTGRESQL + sql_db_container = PostgresContainer(image="postgres:16") + sql_db_container.start() + host = sql_db_container.get_container_host_ip() + port = sql_db_container.get_exposed_port(5432) + user, password, db_id = "test", "test", "test" + info = SQLDBContainerInfo( + adapter=database_type_adapter, + container=sql_db_container, + host=host, + port=port, + user=user, + password=password, + id=db_id) _LOGGER.info( - f"PostgreSQL container started successfully on {cls.get_db_address()}." - ) + "PostgreSQL container started successfully on %s.", info.address) break except Exception as e: _LOGGER.warning( - f"Retry {i + 1}/{cls._sql_client_retries}: Failed to start PostgreSQL container. Reason: {e}" - ) - if i == cls._sql_client_retries - 1: + "Retry %d/%d: Failed to start PostgreSQL container. Reason: %s", + i + 1, + sql_client_retries, + e) + if i == sql_client_retries - 1: _LOGGER.error( - f"Unable to start PostgreSQL container for IO tests after {cls._sql_client_retries} retries. Tests cannot proceed." - ) + "Unable to start PostgreSQL container for IO tests after %d " + "retries. Tests cannot proceed.", + sql_client_retries) raise e - @classmethod - def _stop_sql_db_container(cls): + return info + + @staticmethod + def stop_sql_db_container(sql_db: DbContainer): try: - _LOGGER.info("Stopping PostgreSQL container.") - cls._sql_db_container.stop() - cls._sql_db_container = None + _LOGGER.debug("Stopping PostgreSQL container.") + sql_db.stop() _LOGGER.info("PostgreSQL container stopped successfully.") except Exception as e: _LOGGER.warning( - f"Error encountered while stopping PostgreSQL container: {e}") + "Error encountered while stopping PostgreSQL container: %s", e) + + @staticmethod + def create_table( + table_id: str, + db_url: str, + columns: list[Column], + table_data: list[dict], + metadata: MetaData = MetaData()) -> Engine: + engine = create_engine(db_url) + table = Table(table_id, metadata, *columns) + + # metadata = MetaData() + # Column("id", Integer, primary_key=True), + # Column("name", String, nullable=False), + # Column("quantity", Integer, nullable=False), + # Column("distribution_center_id", Integer, nullable=False), + # Create the table in the database. + metadata.create_all(engine) - @classmethod - def get_db_address(cls): - return f"{cls.sql_db_container_host}:{cls.sql_db_container_port}" + # Insert data into the table. + with engine.connect() as connection: + transaction = connection.begin() + try: + connection.execute(table.insert(), table_data) + transaction.commit() + return engine + except Exception as e: + transaction.rollback() + raise e @pytest.mark.uses_testcontainer -class TestCloudSQLEnrichment(CloudSQLEnrichmentIT): +class TestCloudSQLEnrichment(unittest.TestCase): _table_id = "product_details" + _columns = [ + Column("id", Integer, primary_key=True), + Column("name", String, nullable=False), + Column("quantity", Integer, nullable=False), + Column("distribution_center_id", Integer, nullable=False), + ] _table_data = [ { "id": 1, "name": "A", 'quantity': 2, 'distribution_center_id': 3 @@ -133,44 +192,11 @@ class TestCloudSQLEnrichment(CloudSQLEnrichmentIT): @classmethod def setUpClass(cls): - super(TestCloudSQLEnrichment, cls).setUpClass() - cls.create_table(cls._table_id) + cls.db = CloudSQLEnrichmentTestHelper.start_sql_db_container() + cls._engine = CloudSQLEnrichmentTestHelper.create_table( + cls._table_id, cls.db.url, cls._columns, cls._table_data) cls._cache_client_retries = 3 - @classmethod - def create_table(cls, table_id): - cls._engine = create_engine(cls._get_db_url()) - - # Define the table schema. - metadata = MetaData() - table = Table( - table_id, - metadata, - Column("id", Integer, primary_key=True), - Column("name", String, nullable=False), - Column("quantity", Integer, nullable=False), - Column("distribution_center_id", Integer, nullable=False), - ) - - # Create the table in the database. - metadata.create_all(cls._engine) - - # Insert data into the table. - with cls._engine.connect() as connection: - transaction = connection.begin() - try: - connection.execute(table.insert(), cls._table_data) - transaction.commit() - except Exception as e: - transaction.rollback() - raise e - - @classmethod - def _get_db_url(cls): - dialect = cls.database_type_adapter.to_sqlalchemy_dialect() - db_url = f"{dialect}://{cls.sql_db_user}:{cls.sql_db_password}@{cls.get_db_address()}/{cls.sql_db_id}" - return db_url - @pytest.fixture def cache_container(self): self._start_cache_container() @@ -186,23 +212,24 @@ def _start_cache_container(self): try: self._cache_container = RedisContainer(image="redis:7.2.4") self._cache_container.start() - self._cache_container_host = self._cache_container.get_container_host_ip( - ) - self._cache_container_port = self._cache_container.get_exposed_port( - 6379) + host = self._cache_container.get_container_host_ip() + port = self._cache_container.get_exposed_port(6379) + self._cache_container_host = host + self._cache_container_port = port self._cache_client = self._cache_container.get_client() break except Exception as e: if i == self._cache_client_retries - 1: _LOGGER.error( - f"Unable to start redis container for RRIO tests after {self._cache_client_retries} retries." - ) + "Unable to start redis container for RRIO tests after " + "%d retries.", + self._cache_client_retries) raise e @classmethod def tearDownClass(cls): cls._engine.dispose(close=True) - super(TestCloudSQLEnrichment, cls).tearDownClass() + CloudSQLEnrichmentTestHelper.stop_sql_db_container(cls.db.container) cls._engine = None def test_cloudsql_enrichment(self): @@ -216,11 +243,11 @@ def test_cloudsql_enrichment(self): beam.Row(id=2, name='B'), ] handler = CloudSQLEnrichmentHandler( - database_type_adapter=self.database_type_adapter, - database_address=self.get_db_address(), - database_user=self.sql_db_user, - database_password=self.sql_db_password, - database_id=self.sql_db_id, + database_type_adapter=self.db.adapter, + database_address=self.db.address, + database_user=self.db.user, + database_password=self.db.id, + database_id=self.db.id, table_id=self._table_id, where_clause_template="id = {}", where_clause_fields=fields, @@ -243,11 +270,11 @@ def test_cloudsql_enrichment_batched(self): beam.Row(id=2, name='B'), ] handler = CloudSQLEnrichmentHandler( - database_type_adapter=self.database_type_adapter, - database_address=self.get_db_address(), - database_user=self.sql_db_user, - database_password=self.sql_db_password, - database_id=self.sql_db_id, + database_type_adapter=self.db.adapter, + database_address=self.db.address, + database_user=self.db.user, + database_password=self.db.password, + database_id=self.db.id, table_id=self._table_id, where_clause_template="id = {}", where_clause_fields=fields, @@ -270,11 +297,11 @@ def test_cloudsql_enrichment_batched_multiple_fields(self): beam.Row(id=2, distribution_center_id=1), ] handler = CloudSQLEnrichmentHandler( - database_type_adapter=self.database_type_adapter, - database_address=self.get_db_address(), - database_user=self.sql_db_user, - database_password=self.sql_db_password, - database_id=self.sql_db_id, + database_type_adapter=self.db.adapter, + database_address=self.db.address, + database_user=self.db.user, + database_password=self.db.password, + database_id=self.db.id, table_id=self._table_id, where_clause_template="id = {} AND distribution_center_id = {}", where_clause_fields=fields, @@ -297,11 +324,11 @@ def test_cloudsql_enrichment_with_query_fn(self): ] fn = functools.partial(query_fn, self._table_id) handler = CloudSQLEnrichmentHandler( - database_type_adapter=self.database_type_adapter, - database_address=self.get_db_address(), - database_user=self.sql_db_user, - database_password=self.sql_db_password, - database_id=self.sql_db_id, + database_type_adapter=self.db.adapter, + database_address=self.db.address, + database_user=self.db.user, + database_password=self.db.password, + database_id=self.db.id, query_fn=fn) with TestPipeline(is_integration_test=True) as test_pipeline: pcoll = (test_pipeline | beam.Create(requests) | Enrichment(handler)) @@ -318,11 +345,11 @@ def test_cloudsql_enrichment_with_condition_value_fn(self): beam.Row(id=2, name='B'), ] handler = CloudSQLEnrichmentHandler( - database_type_adapter=self.database_type_adapter, - database_address=self.get_db_address(), - database_user=self.sql_db_user, - database_password=self.sql_db_password, - database_id=self.sql_db_id, + database_type_adapter=self.db.adapter, + database_address=self.db.address, + database_user=self.db.user, + database_password=self.db.password, + database_id=self.db.id, table_id=self._table_id, where_clause_template="id = {}", where_clause_value_fn=where_clause_value_fn, @@ -339,11 +366,11 @@ def test_cloudsql_enrichment_table_nonexistent_runtime_error_raised(self): beam.Row(id=2, name='B'), ] handler = CloudSQLEnrichmentHandler( - database_type_adapter=self.database_type_adapter, - database_address=self.get_db_address(), - database_user=self.sql_db_user, - database_password=self.sql_db_password, - database_id=self.sql_db_id, + database_type_adapter=self.db.adapter, + database_address=self.db.address, + database_user=self.db.user, + database_password=self.db.password, + database_id=self.db.id, table_id=self._table_id, where_clause_template="id = {}", where_clause_value_fn=where_clause_value_fn, @@ -369,11 +396,11 @@ def test_cloudsql_enrichment_with_redis(self): beam.Row(id=2, name="B", quantity=3, distribution_center_id=1) ] handler = CloudSQLEnrichmentHandler( - database_type_adapter=self.database_type_adapter, - database_address=self.get_db_address(), - database_user=self.sql_db_user, - database_password=self.sql_db_password, - database_id=self.sql_db_id, + database_type_adapter=self.db.adapter, + database_address=self.db.address, + database_user=self.db.user, + database_password=self.db.password, + database_id=self.db.id, table_id=self._table_id, where_clause_template="id = {}", where_clause_value_fn=where_clause_value_fn, diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_test.py index 2fa74bf33233..888886479c75 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_test.py @@ -20,9 +20,12 @@ # pylint: disable=ungrouped-imports try: - from apache_beam.transforms.enrichment_handlers.cloudsql import CloudSQLEnrichmentHandler, DatabaseTypeAdapter - from apache_beam.transforms.enrichment_handlers.cloudsql_it_test import where_clause_value_fn - from apache_beam.transforms.enrichment_handlers.cloudsql_it_test import query_fn + from apache_beam.transforms.enrichment_handlers.cloudsql import ( + CloudSQLEnrichmentHandler, DatabaseTypeAdapter) + from apache_beam.transforms.enrichment_handlers.cloudsql_it_test import ( + query_fn, + where_clause_value_fn, + ) except ImportError: raise unittest.SkipTest('Google Cloud SQL dependencies are not installed.') From 4fa1830837153c5969861c59e724bcbc661fe641 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 13 Apr 2025 00:08:02 +0000 Subject: [PATCH 10/83] website: add missing `enrichment-cloudsql.md` --- .../python/elementwise/enrichment-cloudsql.md | 63 +++++++++++++++++++ 1 file changed, 63 insertions(+) create mode 100644 website/www/site/content/en/documentation/transforms/python/elementwise/enrichment-cloudsql.md diff --git a/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment-cloudsql.md b/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment-cloudsql.md new file mode 100644 index 000000000000..a8c5de1c2694 --- /dev/null +++ b/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment-cloudsql.md @@ -0,0 +1,63 @@ +--- +title: "Enrichment with CloudSQL" +--- + + +# Use CloudSQL to enrich data + +{{< localstorage language language-py >}} + + + + + +
+ + {{< button-pydoc path="apache_beam.transforms.enrichment_handlers.cloudsql" class="CloudSQLEnrichmentHandler" >}} + +
+ +In Apache Beam and later versions, the enrichment transform includes +a built-in enrichment handler for +[CloudSQL](https://cloud.google.com/sql/docs). +The following example demonstrates how to create a pipeline that use the enrichment transform with the [`CloudSQLEnrichmentHandler`](https://beam.apache.org/releases/pydoc/current/apache_beam.transforms.enrichment_handlers.cloudsql.html#apache_beam.transforms.enrichment_handlers.cloudsql.CloudSQLEnrichmentHandler) handler. + +The data in the CloudSQL PostgreSQL products table instance follows this format: + +{{< table >}} +| product_id | name | quantity | region_id | +|:----------:|:----:|:--------:|:---------:| +| 1 | A | 2 | 3 | +| 2 | B | 3 | 1 | +| 3 | C | 10 | 4 | +{{< /table >}} + + +{{< highlight language="py" >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py" enrichment_with_cloudsql >}} +{{}} + +{{< paragraph class="notebook-skip" >}} +Output: +{{< /paragraph >}} +{{< highlight class="notebook-skip" >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py" enrichment_with_cloudsql >}} +{{< /highlight >}} + +## Related transforms + +Not applicable. + +{{< button-pydoc path="apache_beam.transforms.enrichment_handlers.cloudsql" class="CloudSQLEnrichmentHandler" >}} \ No newline at end of file From 4251a7677b9514b324db0c7afa87c6636fb6df4d Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 13 Apr 2025 00:19:26 +0000 Subject: [PATCH 11/83] nits: remove commented code --- .../transforms/enrichment_handlers/cloudsql_it_test.py | 7 ------- 1 file changed, 7 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py index 58ce8dce5263..599528198431 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py @@ -133,13 +133,6 @@ def create_table( metadata: MetaData = MetaData()) -> Engine: engine = create_engine(db_url) table = Table(table_id, metadata, *columns) - - # metadata = MetaData() - # Column("id", Integer, primary_key=True), - # Column("name", String, nullable=False), - # Column("quantity", Integer, nullable=False), - # Column("distribution_center_id", Integer, nullable=False), - # Create the table in the database. metadata.create_all(engine) # Insert data into the table. From ab7cf7b6711430adacb97c6740d48049f15d439e Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 13 Apr 2025 00:28:58 +0000 Subject: [PATCH 12/83] sdks/python: remove `SQL_TABLE_ID` env variable --- .../examples/snippets/transforms/elementwise/enrichment.py | 2 +- .../snippets/transforms/elementwise/enrichment_test.py | 5 ++--- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py index 13a0ad2e226f..05ecddbef54d 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py @@ -64,7 +64,7 @@ def enrichment_with_cloudsql(): database_user = os.environ.get("SQL_DB_USER") database_password = os.environ.get("SQL_DB_PASSWORD") database_id = os.environ.get("SQL_DB_ID") - table_id = os.environ.get("SQL_TABLE_ID") + table_id = "products" where_clause_template = "product_id = {}" where_clause_fields = ["product_id"] diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py index 82d9caaf3a6b..c62da660551a 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py @@ -114,6 +114,7 @@ def test_enrichment_with_vertex_ai_legacy(self, mock_stdout): self.assertEqual(output, expected) def pre_cloudsql_enrichment_test(self): + table_id ="products" columns = [ Column("product_id", Integer, primary_key=True), Column("name", String, nullable=False), @@ -138,10 +139,9 @@ def pre_cloudsql_enrichment_test(self): os.environ['SQL_DB_PASSWORD'] = db.password os.environ['SQL_DB_ID'] = db.id os.environ['SQL_DB_URL'] = db.url - os.environ['SQL_TABLE_ID'] = "products" engine = CloudSQLEnrichmentTestHelper.create_table( - table_id=os.environ.get("SQL_TABLE_ID"), db_url=os.environ.get("SQL_DB_URL"), + table_id=table_id, columns=columns, table_data=table_data) return db, engine @@ -156,7 +156,6 @@ def post_cloudsql_enrichment_test( os.environ.pop('SQL_DB_PASSWORD', None) os.environ.pop('SQL_DB_ID', None) os.environ.pop('SQL_DB_URL', None) - os.environ.pop('SQL_TABLE_ID', None) if __name__ == '__main__': From 669d2d4d88b45378c2c80c31e67a37c7959d1dd1 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 13 Apr 2025 00:47:56 +0000 Subject: [PATCH 13/83] sdks/python: fix formatting issues --- .../examples/snippets/transforms/elementwise/enrichment_test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py index c62da660551a..7f2987b240a1 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py @@ -114,7 +114,7 @@ def test_enrichment_with_vertex_ai_legacy(self, mock_stdout): self.assertEqual(output, expected) def pre_cloudsql_enrichment_test(self): - table_id ="products" + table_id = "products" columns = [ Column("product_id", Integer, primary_key=True), Column("name", String, nullable=False), From 720933542cad6a63c19c684ef497bbdfe43283cd Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Wed, 21 May 2025 04:05:44 +0000 Subject: [PATCH 14/83] sdks/python: address claudevdm feedback (2) --- .../transforms/elementwise/enrichment.py | 10 +- .../transforms/elementwise/enrichment_test.py | 7 +- .../enrichment_handlers/cloudsql.py | 223 ++++++++++-------- .../enrichment_handlers/cloudsql_it_test.py | 184 +++++++++++---- .../enrichment_handlers/cloudsql_test.py | 135 +++++++++-- 5 files changed, 389 insertions(+), 170 deletions(-) diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py index 05ecddbef54d..0190b12b700d 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py @@ -56,7 +56,7 @@ def enrichment_with_cloudsql(): import apache_beam as beam from apache_beam.transforms.enrichment import Enrichment from apache_beam.transforms.enrichment_handlers.cloudsql import ( - CloudSQLEnrichmentHandler, DatabaseTypeAdapter) + CloudSQLEnrichmentHandler, DatabaseTypeAdapter, TableFieldsQueryConfig) import os database_type_adapter = DatabaseTypeAdapter[os.environ.get("SQL_DB_TYPE")] @@ -74,6 +74,11 @@ def enrichment_with_cloudsql(): beam.Row(product_id=3, name='C'), ] + query_config = TableFieldsQueryConfig( + table_id=table_id, + where_clause_template=where_clause_template, + where_clause_fields=where_clause_fields) + cloudsql_handler = CloudSQLEnrichmentHandler( database_type_adapter=database_type_adapter, database_address=database_address, @@ -81,8 +86,7 @@ def enrichment_with_cloudsql(): database_password=database_password, database_id=database_id, table_id=table_id, - where_clause_template=where_clause_template, - where_clause_fields=where_clause_fields) + query_config=query_config) with beam.Pipeline() as p: _ = ( p diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py index 7f2987b240a1..426f96d58b8e 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py @@ -32,6 +32,8 @@ enrichment_with_bigtable, enrichment_with_vertex_ai_legacy) from apache_beam.examples.snippets.transforms.elementwise.enrichment import ( enrichment_with_vertex_ai, enrichment_with_cloudsql) + from apache_beam.transforms.enrichment_handlers.cloudsql import ( + DatabaseTypeAdapter) from apache_beam.transforms.enrichment_handlers.cloudsql_it_test import ( CloudSQLEnrichmentTestHelper, SQLDBContainerInfo) from apache_beam.io.requestresponse import RequestResponseIO @@ -132,7 +134,8 @@ def pre_cloudsql_enrichment_test(self): "product_id": 3, "name": "C", 'quantity': 10, 'region_id': 4 }, ] - db = CloudSQLEnrichmentTestHelper.start_sql_db_container() + db_adapter = DatabaseTypeAdapter.POSTGRESQL + db = CloudSQLEnrichmentTestHelper.start_sql_db_container(db_adapter) os.environ['SQL_DB_TYPE'] = db.adapter.name os.environ['SQL_DB_ADDRESS'] = db.address os.environ['SQL_DB_USER'] = db.user @@ -149,7 +152,7 @@ def pre_cloudsql_enrichment_test(self): def post_cloudsql_enrichment_test( self, db: SQLDBContainerInfo, engine: Engine): engine.dispose(close=True) - CloudSQLEnrichmentTestHelper.stop_sql_db_container(db.container) + CloudSQLEnrichmentTestHelper.stop_sql_db_container(db) os.environ.pop('SQL_DB_TYPE', None) os.environ.pop('SQL_DB_ADDRESS', None) os.environ.pop('SQL_DB_USER', None) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py index 4f2ccea3575c..c676f92ecdbd 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py @@ -16,10 +16,13 @@ # from collections.abc import Callable from collections.abc import Mapping +from dataclasses import dataclass from enum import Enum from typing import Any +from typing import List from typing import Optional from typing import Union +from typing import cast from sqlalchemy import create_engine from sqlalchemy import text @@ -31,43 +34,75 @@ ConditionValueFn = Callable[[beam.Row], list[Any]] -def _validate_cloudsql_metadata( - table_id, - where_clause_template, - where_clause_fields, - where_clause_value_fn, - query_fn): - if query_fn: - if any([table_id, - where_clause_template, - where_clause_fields, - where_clause_value_fn]): +@dataclass +class CustomQueryConfig: + """Configuration for using a custom query function.""" + query_fn: QueryFn + + +@dataclass +class TableFieldsQueryConfig: + """Configuration for using table name, where clause, and field names.""" + table_id: str + where_clause_template: str + where_clause_fields: List[str] + + +@dataclass +class TableFunctionQueryConfig: + """Configuration for using table name, where clause, and a value function.""" + table_id: str + where_clause_template: str + where_clause_value_fn: ConditionValueFn + + +QueryConfig = Union[CustomQueryConfig, + TableFieldsQueryConfig, + TableFunctionQueryConfig] + + +def _validate_query_config(query_config: QueryConfig): + """Validates the provided query configuration.""" + if isinstance(query_config, CustomQueryConfig): + if not query_config.query_fn: + raise ValueError("CustomQueryConfig must provide a valid query_fn") + elif isinstance(query_config, + (TableFieldsQueryConfig, TableFunctionQueryConfig)): + if not query_config.table_id or not query_config.where_clause_template: raise ValueError( - "Please provide either `query_fn` or the parameters `table_id`, " - "`where_clause_template`, and " - "`where_clause_fields/where_clause_value_fn` together.") + "TableFieldsQueryConfig and " + + "TableFunctionQueryConfig must provide table_id " + + "and where_clause_template") + + is_table_fields = isinstance(query_config, TableFieldsQueryConfig) + if is_table_fields: + table_fields_config = cast(TableFieldsQueryConfig, query_config) + if not table_fields_config.where_clause_fields: + raise ValueError( + "TableFieldsQueryConfig must provide non-empty " + + "where_clause_fields") + + is_table_function = isinstance(query_config, TableFunctionQueryConfig) + if is_table_function: + table_function_config = cast(TableFunctionQueryConfig, query_config) + if not table_function_config.where_clause_value_fn: + raise ValueError( + "TableFunctionQueryConfig must provide " + "where_clause_value_fn") else: - if not (table_id and where_clause_template): - raise ValueError( - "Please provide either `query_fn` or the parameters " - "`table_id` and `where_clause_template` together.") - if (bool(where_clause_fields) == bool(where_clause_value_fn)): - raise ValueError( - "Please provide exactly one of `where_clause_fields` or " - "`where_clause_value_fn`.") + raise ValueError("Invalid query_config type provided") class DatabaseTypeAdapter(Enum): POSTGRESQL = "psycopg2" MYSQL = "pymysql" - SQLSERVER = "pytds" + SQLSERVER = "pymysql" def to_sqlalchemy_dialect(self): + """Map the adapter type to its corresponding SQLAlchemy dialect. + + Returns: + str: SQLAlchemy dialect string. """ - Map the adapter type to its corresponding SQLAlchemy dialect. - Returns: - str: SQLAlchemy dialect string. - """ if self == DatabaseTypeAdapter.POSTGRESQL: return f"postgresql+{self.value}" elif self == DatabaseTypeAdapter.MYSQL: @@ -79,16 +114,15 @@ def to_sqlalchemy_dialect(self): class CloudSQLEnrichmentHandler(EnrichmentSourceHandler[beam.Row, beam.Row]): - """ - Enrichment handler for Cloud SQL databases. + """Enrichment handler for Cloud SQL databases. This handler is designed to work with the :class:`apache_beam.transforms.enrichment.Enrichment` transform. - To use this handler, you need to provide either of the following combinations: - * `table_id`, `where_clause_template`, `where_clause_fields` - * `table_id`, `where_clause_template`, `where_clause_value_fn` - * `query_fn` + To use this handler, you need to provide one of the following query configs: + * CustomQueryConfig - For providing a custom query function + * TableFieldsQueryConfig - For specifying table, where clause, and fields + * TableFunctionQueryConfig - For specifying table, where clause, and val fn By default, the handler retrieves all columns from the specified table. To limit the columns, use the `column_names` parameter to specify @@ -99,7 +133,7 @@ class CloudSQLEnrichmentHandler(EnrichmentSourceHandler[beam.Row, beam.Row]): These values control the batching behavior in the :class:`apache_beam.transforms.utils.BatchElements` transform. - NOTE: Batching is not supported when using the `query_fn` parameter. + NOTE: Batching is not supported when using the CustomQueryConfig. """ def __init__( self, @@ -109,11 +143,7 @@ def __init__( database_password: str, database_id: str, *, - table_id: str = "", - where_clause_template: str = "", - where_clause_fields: Optional[list[str]] = None, - where_clause_value_fn: Optional[ConditionValueFn] = None, - query_fn: Optional[QueryFn] = None, + query_config: QueryConfig, column_names: Optional[list[str]] = None, min_batch_size: int = 1, max_batch_size: int = 10000, @@ -127,12 +157,9 @@ def __init__( database_user='user', database_password='password', database_id='my_database', - table_id='my_table', - where_clause_template="id = '{}'", - where_clause_fields=['id'], + query_config=TableFieldsQueryConfig('my_table',"id = '{}'",['id']), min_batch_size=2, - max_batch_size=100 - ) + max_batch_size=100) Args: database_type_adapter: Adapter to handle specific database type operations @@ -143,18 +170,10 @@ def __init__( database_user (str): Username for accessing the database. database_password (str): Password for accessing the database. database_id (str): Identifier for the database to query. - table_id (str): Name of the table to query in the Cloud SQL database. - where_clause_template (str): A template string for the `WHERE` clause - in the SQL query with placeholders (`{}`) for dynamic filtering - based on input data. - where_clause_fields (Optional[list[str]]): List of field names from the - input `beam.Row` used to construct the `WHERE` clause if - `where_clause_value_fn` is not provided. - where_clause_value_fn (Optional[Callable[[beam.Row], Any]]): Function that - takes a `beam.Row` and returns a list of values to populate the - placeholders `{}` in the `WHERE` clause. - query_fn (Optional[Callable[[beam.Row], str]]): Function that takes a - `beam.Row` and returns a complete SQL query string. + query_config: Configuration for database queries. Must be one of: + * CustomQueryConfig: For providing a custom query function + * TableFieldsQueryConfig: specifies table, where clause, and field names + * TableFunctionQueryConfig: specifies table, where clause, and val func column_names (Optional[list[str]]): List of column names to select from the Cloud SQL table. If not provided, all columns (`*`) are selected. min_batch_size (int): Minimum number of rows to batch together when @@ -171,31 +190,22 @@ def __init__( * Ensure that the database user has the necessary permissions to query the specified table. """ - _validate_cloudsql_metadata( - table_id, - where_clause_template, - where_clause_fields, - where_clause_value_fn, - query_fn) + _validate_query_config(query_config) self._database_type_adapter = database_type_adapter self._database_id = database_id self._database_user = database_user self._database_password = database_password self._database_address = database_address - self._table_id = table_id - self._where_clause_template = where_clause_template - self._where_clause_value_fn = where_clause_value_fn - self._query_fn = query_fn - fields = where_clause_fields if where_clause_fields else [] - self._where_clause_fields = fields + self._query_config = query_config self._column_names = ",".join(column_names) if column_names else "*" - self.query_template = ( - f"SELECT {self._column_names} " - f"FROM {self._table_id} " - f"WHERE {self._where_clause_template}") self.kwargs = kwargs self._batching_kwargs = {} - if not query_fn: + table_query_configs = (TableFieldsQueryConfig, TableFunctionQueryConfig) + if isinstance(query_config, table_query_configs): + self.query_template = ( + f"SELECT {self._column_names} " + f"FROM {query_config.table_id} " + f"WHERE {query_config.where_clause_template}") self._batching_kwargs['min_batch_size'] = min_batch_size self._batching_kwargs['max_batch_size'] = max_batch_size @@ -233,20 +243,26 @@ def __call__(self, request: Union[beam.Row, list[beam.Row]], *args, **kwargs): # For multiple requests in the batch, combine the WHERE clause conditions # using 'OR' and update the query template to handle all requests. - if batch_size > 1: + table_query_configs = (TableFieldsQueryConfig, TableFunctionQueryConfig) + if batch_size > 1 and isinstance(self._query_config, table_query_configs): where_clause_template_batched = ' OR '.join( - [fr'({self._where_clause_template})'] * batch_size) + [fr'({self._query_config.where_clause_template})'] * batch_size) raw_query = self.query_template.replace( - self._where_clause_template, where_clause_template_batched) + self._query_config.where_clause_template, + where_clause_template_batched) # Extract where_clause_fields values and map the generated request key to # the original request object. for req in request: request_dict = req._asdict() try: - current_values = ( - self._where_clause_value_fn(req) if self._where_clause_value_fn - else [request_dict[field] for field in self._where_clause_fields]) + if isinstance(self._query_config, TableFunctionQueryConfig): + current_values = self._query_config.where_clause_value_fn(req) + elif isinstance(self._query_config, TableFieldsQueryConfig): + current_values = [ + request_dict[field] + for field in self._query_config.where_clause_fields + ] except KeyError as e: raise KeyError( "Make sure the values passed in `where_clause_fields` are the " @@ -266,14 +282,17 @@ def __call__(self, request: Union[beam.Row, list[beam.Row]], *args, **kwargs): return responses else: request_dict = request._asdict() - if self._query_fn: - query = self._query_fn(request) + if isinstance(self._query_config, CustomQueryConfig): + query = self._query_config.query_fn(request) else: try: - values = ( - self._where_clause_value_fn(request) - if self._where_clause_value_fn else - [request_dict[field] for field in self._where_clause_fields]) + if isinstance(self._query_config, TableFunctionQueryConfig): + values = self._query_config.where_clause_value_fn(request) + elif isinstance(self._query_config, TableFieldsQueryConfig): + values = [ + request_dict[field] + for field in self._query_config.where_clause_fields + ] except KeyError as e: raise KeyError( "Make sure the values passed in `where_clause_fields` are the " @@ -283,14 +302,14 @@ def __call__(self, request: Union[beam.Row, list[beam.Row]], *args, **kwargs): return request, beam.Row(**response_dict) def create_row_key(self, row: beam.Row): - if self._where_clause_value_fn: - return tuple(self._where_clause_value_fn(row)) - if self._where_clause_fields: + if isinstance(self._query_config, TableFunctionQueryConfig): + return tuple(self._query_config.where_clause_value_fn(row)) + if isinstance(self._query_config, TableFieldsQueryConfig): row_dict = row._asdict() return ( tuple( row_dict[where_clause_field] - for where_clause_field in self._where_clause_fields)) + for where_clause_field in self._query_config.where_clause_fields)) raise ValueError( "Either where_clause_fields or where_clause_value_fn must be specified") @@ -300,14 +319,24 @@ def __exit__(self, exc_type, exc_val, exc_tb): self._engine, self._connection = None, None def get_cache_key(self, request: Union[beam.Row, list[beam.Row]]): + if isinstance(self._query_config, CustomQueryConfig): + raise NotImplementedError( + "Caching is not supported for CustomQueryConfig. " + "Consider using TableFieldsQueryConfig or " + + "TableFunctionQueryConfig instead.") + if isinstance(request, list): cache_keys = [] for req in request: req_dict = req._asdict() try: - current_values = ( - self._where_clause_value_fn(req) if self._where_clause_value_fn - else [req_dict[field] for field in self._where_clause_fields]) + if isinstance(self._query_config, TableFunctionQueryConfig): + current_values = self._query_config.where_clause_value_fn(req) + elif isinstance(self._query_config, TableFieldsQueryConfig): + current_values = [ + req_dict[field] + for field in self._query_config.where_clause_fields + ] key = ";".join(["%s"] * len(current_values)) cache_keys.extend([key % tuple(current_values)]) except KeyError as e: @@ -318,9 +347,13 @@ def get_cache_key(self, request: Union[beam.Row, list[beam.Row]]): else: req_dict = request._asdict() try: - current_values = ( - self._where_clause_value_fn(request) if self._where_clause_value_fn - else [req_dict[field] for field in self._where_clause_fields]) + if isinstance(self._query_config, TableFunctionQueryConfig): + current_values = self._query_config.where_clause_value_fn(request) + else: # TableFieldsQueryConfig + current_values = [ + req_dict[field] + for field in self._query_config.where_clause_fields + ] key = ";".join(["%s"] * len(current_values)) cache_key = key % tuple(current_values) except KeyError as e: diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py index 599528198431..e989be7079ba 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py @@ -33,6 +33,8 @@ try: from testcontainers.core.generic import DbContainer from testcontainers.postgres import PostgresContainer + from testcontainers.mysql import MySqlContainer + from testcontainers.mssql import SqlServerContainer from testcontainers.redis import RedisContainer from sqlalchemy import ( create_engine, MetaData, Table, Column, Integer, String, Engine) @@ -40,7 +42,9 @@ from apache_beam.transforms.enrichment_handlers.cloudsql import ( CloudSQLEnrichmentHandler, DatabaseTypeAdapter, - ) + CustomQueryConfig, + TableFieldsQueryConfig, + TableFunctionQueryConfig) except ImportError: raise unittest.SkipTest('Google Cloud SQL dependencies are not installed.') @@ -78,18 +82,36 @@ def url(self) -> str: class CloudSQLEnrichmentTestHelper: @staticmethod def start_sql_db_container( + database_type: DatabaseTypeAdapter, sql_client_retries=3) -> Optional[SQLDBContainerInfo]: info = None for i in range(sql_client_retries): try: - database_type_adapter = DatabaseTypeAdapter.POSTGRESQL - sql_db_container = PostgresContainer(image="postgres:16") - sql_db_container.start() - host = sql_db_container.get_container_host_ip() - port = sql_db_container.get_exposed_port(5432) - user, password, db_id = "test", "test", "test" + if database_type == DatabaseTypeAdapter.POSTGRESQL: + sql_db_container = PostgresContainer(image="postgres:16") + sql_db_container.start() + host = sql_db_container.get_container_host_ip() + port = sql_db_container.get_exposed_port(5432) + user, password, db_id = "test", "test", "test" + + elif database_type == DatabaseTypeAdapter.MYSQL: + sql_db_container = MySqlContainer(image="mysql:8.0") + sql_db_container.start() + host = sql_db_container.get_container_host_ip() + port = sql_db_container.get_exposed_port(3306) + user, password, db_id = "test", "test", "test" + + elif database_type == DatabaseTypeAdapter.SQLSERVER: + sql_db_container = SqlServerContainer() + sql_db_container.start() + host = sql_db_container.get_container_host_ip() + port = sql_db_container.get_exposed_port(1433) + user, password, db_id = "sa", "A_Str0ng_Required_Password", "tempdb" + else: + raise ValueError(f"Unsupported database type: {database_type}") + info = SQLDBContainerInfo( - adapter=database_type_adapter, + adapter=database_type, container=sql_db_container, host=host, port=port, @@ -97,32 +119,38 @@ def start_sql_db_container( password=password, id=db_id) _LOGGER.info( - "PostgreSQL container started successfully on %s.", info.address) + "%s container started successfully on %s.", + database_type.name, + info.address) break except Exception as e: _LOGGER.warning( - "Retry %d/%d: Failed to start PostgreSQL container. Reason: %s", + "Retry %d/%d: Failed to start %s container. Reason: %s", i + 1, sql_client_retries, + database_type.name, e) if i == sql_client_retries - 1: _LOGGER.error( - "Unable to start PostgreSQL container for IO tests after %d " + "Unable to start %s container for I/O tests after %d " "retries. Tests cannot proceed.", + database_type.name, sql_client_retries) raise e return info @staticmethod - def stop_sql_db_container(sql_db: DbContainer): + def stop_sql_db_container(db_info: SQLDBContainerInfo): try: - _LOGGER.debug("Stopping PostgreSQL container.") - sql_db.stop() - _LOGGER.info("PostgreSQL container stopped successfully.") + _LOGGER.debug("Stopping %s container.", db_info.adapter.name) + db_info.container.stop() + _LOGGER.info("%s container stopped successfully.", db_info.adapter.name) except Exception as e: _LOGGER.warning( - "Error encountered while stopping PostgreSQL container: %s", e) + "Error encountered while stopping %s container: %s", + db_info.adapter.name, + e) @staticmethod def create_table( @@ -141,21 +169,23 @@ def create_table( try: connection.execute(table.insert(), table_data) transaction.commit() - return engine except Exception as e: transaction.rollback() raise e + return engine + + +def init_db_type(db_type): + def wrapper(cls): + cls.db_type = db_type + return cls + + return wrapper + @pytest.mark.uses_testcontainer -class TestCloudSQLEnrichment(unittest.TestCase): - _table_id = "product_details" - _columns = [ - Column("id", Integer, primary_key=True), - Column("name", String, nullable=False), - Column("quantity", Integer, nullable=False), - Column("distribution_center_id", Integer, nullable=False), - ] +class BaseTestCloudSQLEnrichment(unittest.TestCase): _table_data = [ { "id": 1, "name": "A", 'quantity': 2, 'distribution_center_id': 3 @@ -182,14 +212,29 @@ class TestCloudSQLEnrichment(unittest.TestCase): "id": 8, "name": "D", 'quantity': 4, 'distribution_center_id': 1 }, ] + db = None + _engine = None @classmethod def setUpClass(cls): - cls.db = CloudSQLEnrichmentTestHelper.start_sql_db_container() + if not hasattr(cls, 'db_type'): + # Skip setup for the base class. + raise unittest.SkipTest("Base class - no db_type defined") + cls.db = CloudSQLEnrichmentTestHelper.start_sql_db_container(cls.db_type) cls._engine = CloudSQLEnrichmentTestHelper.create_table( - cls._table_id, cls.db.url, cls._columns, cls._table_data) + cls._table_id, cls.db.url, cls.get_columns(), cls._table_data) cls._cache_client_retries = 3 + @classmethod + def get_columns(cls): + """Returns fresh column objects each time it's called.""" + return [ + Column("id", Integer, primary_key=True), + Column("name", String(255), nullable=False), + Column("quantity", Integer, nullable=False), + Column("distribution_center_id", Integer, nullable=False), + ] + @pytest.fixture def cache_container(self): self._start_cache_container() @@ -222,7 +267,7 @@ def _start_cache_container(self): @classmethod def tearDownClass(cls): cls._engine.dispose(close=True) - CloudSQLEnrichmentTestHelper.stop_sql_db_container(cls.db.container) + CloudSQLEnrichmentTestHelper.stop_sql_db_container(cls.db) cls._engine = None def test_cloudsql_enrichment(self): @@ -235,15 +280,19 @@ def test_cloudsql_enrichment(self): beam.Row(id=1, name='A'), beam.Row(id=2, name='B'), ] + + query_config = TableFieldsQueryConfig( + table_id=self._table_id, + where_clause_template="id = {}", + where_clause_fields=fields) + handler = CloudSQLEnrichmentHandler( database_type_adapter=self.db.adapter, database_address=self.db.address, database_user=self.db.user, database_password=self.db.id, database_id=self.db.id, - table_id=self._table_id, - where_clause_template="id = {}", - where_clause_fields=fields, + query_config=query_config, min_batch_size=1, max_batch_size=100, ) @@ -262,15 +311,19 @@ def test_cloudsql_enrichment_batched(self): beam.Row(id=1, name='A'), beam.Row(id=2, name='B'), ] + + query_config = TableFieldsQueryConfig( + table_id=self._table_id, + where_clause_template="id = {}", + where_clause_fields=fields) + handler = CloudSQLEnrichmentHandler( database_type_adapter=self.db.adapter, database_address=self.db.address, database_user=self.db.user, database_password=self.db.password, database_id=self.db.id, - table_id=self._table_id, - where_clause_template="id = {}", - where_clause_fields=fields, + query_config=query_config, min_batch_size=2, max_batch_size=100, ) @@ -289,15 +342,19 @@ def test_cloudsql_enrichment_batched_multiple_fields(self): beam.Row(id=1, distribution_center_id=3), beam.Row(id=2, distribution_center_id=1), ] + + query_config = TableFieldsQueryConfig( + table_id=self._table_id, + where_clause_template="id = {} AND distribution_center_id = {}", + where_clause_fields=fields) + handler = CloudSQLEnrichmentHandler( database_type_adapter=self.db.adapter, database_address=self.db.address, database_user=self.db.user, database_password=self.db.password, database_id=self.db.id, - table_id=self._table_id, - where_clause_template="id = {} AND distribution_center_id = {}", - where_clause_fields=fields, + query_config=query_config, min_batch_size=8, max_batch_size=100, ) @@ -316,13 +373,16 @@ def test_cloudsql_enrichment_with_query_fn(self): beam.Row(id=2, name='B'), ] fn = functools.partial(query_fn, self._table_id) + + query_config = CustomQueryConfig(query_fn=fn) + handler = CloudSQLEnrichmentHandler( database_type_adapter=self.db.adapter, database_address=self.db.address, database_user=self.db.user, database_password=self.db.password, database_id=self.db.id, - query_fn=fn) + query_config=query_config) with TestPipeline(is_integration_test=True) as test_pipeline: pcoll = (test_pipeline | beam.Create(requests) | Enrichment(handler)) @@ -337,15 +397,19 @@ def test_cloudsql_enrichment_with_condition_value_fn(self): beam.Row(id=1, name='A'), beam.Row(id=2, name='B'), ] + + query_config = TableFunctionQueryConfig( + table_id=self._table_id, + where_clause_template="id = {}", + where_clause_value_fn=where_clause_value_fn) + handler = CloudSQLEnrichmentHandler( database_type_adapter=self.db.adapter, database_address=self.db.address, database_user=self.db.user, database_password=self.db.password, database_id=self.db.id, - table_id=self._table_id, - where_clause_template="id = {}", - where_clause_value_fn=where_clause_value_fn, + query_config=query_config, min_batch_size=2, max_batch_size=100) with TestPipeline(is_integration_test=True) as test_pipeline: @@ -358,15 +422,19 @@ def test_cloudsql_enrichment_table_nonexistent_runtime_error_raised(self): beam.Row(id=1, name='A'), beam.Row(id=2, name='B'), ] + + query_config = TableFunctionQueryConfig( + table_id=self._table_id, + where_clause_template="id = {}", + where_clause_value_fn=where_clause_value_fn) + handler = CloudSQLEnrichmentHandler( database_type_adapter=self.db.adapter, database_address=self.db.address, database_user=self.db.user, database_password=self.db.password, database_id=self.db.id, - table_id=self._table_id, - where_clause_template="id = {}", - where_clause_value_fn=where_clause_value_fn, + query_config=query_config, column_names=["wrong_column"], ) with self.assertRaises(RuntimeError): @@ -388,15 +456,19 @@ def test_cloudsql_enrichment_with_redis(self): beam.Row(id=1, name="A", quantity=2, distribution_center_id=3), beam.Row(id=2, name="B", quantity=3, distribution_center_id=1) ] + + query_config = TableFunctionQueryConfig( + table_id=self._table_id, + where_clause_template="id = {}", + where_clause_value_fn=where_clause_value_fn) + handler = CloudSQLEnrichmentHandler( database_type_adapter=self.db.adapter, database_address=self.db.address, database_user=self.db.user, database_password=self.db.password, database_id=self.db.id, - table_id=self._table_id, - where_clause_template="id = {}", - where_clause_value_fn=where_clause_value_fn, + query_config=query_config, min_batch_size=2, max_batch_size=100) with TestPipeline(is_integration_test=True) as test_pipeline: @@ -433,5 +505,23 @@ def test_cloudsql_enrichment_with_redis(self): CloudSQLEnrichmentHandler.__call__ = actual +@init_db_type(DatabaseTypeAdapter.POSTGRESQL) +@pytest.mark.uses_testcontainer +class TestCloudSQLEnrichmentPostgres(BaseTestCloudSQLEnrichment): + _table_id = "product_details_pg" + + +@init_db_type(DatabaseTypeAdapter.MYSQL) +@pytest.mark.uses_testcontainer +class TestCloudSQLEnrichmentMySQL(BaseTestCloudSQLEnrichment): + _table_id = "product_details_mysql" + + +@init_db_type(DatabaseTypeAdapter.SQLSERVER) +@pytest.mark.uses_testcontainer +class TestCloudSQLEnrichmentSQLServer(BaseTestCloudSQLEnrichment): + _table_id = "product_details_mssql" + + if __name__ == "__main__": unittest.main() diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_test.py index 888886479c75..f3da541cd89a 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_test.py @@ -21,7 +21,11 @@ # pylint: disable=ungrouped-imports try: from apache_beam.transforms.enrichment_handlers.cloudsql import ( - CloudSQLEnrichmentHandler, DatabaseTypeAdapter) + CloudSQLEnrichmentHandler, + DatabaseTypeAdapter, + CustomQueryConfig, + TableFieldsQueryConfig, + TableFunctionQueryConfig) from apache_beam.transforms.enrichment_handlers.cloudsql_it_test import ( query_fn, where_clause_value_fn, @@ -32,25 +36,40 @@ class TestCloudSQLEnrichment(unittest.TestCase): @parameterized.expand([ - ("", "", [], None, None, 1, 2), - ("table", "", ["id"], where_clause_value_fn, None, 2, 10), - ("table", "id='{}'", ["id"], where_clause_value_fn, None, 2, 10), - ("table", "id='{}'", ["id"], None, query_fn, 2, 10), + # Empty TableFieldsQueryConfig. + ( + TableFieldsQueryConfig( + table_id="", where_clause_template="", where_clause_fields=[]), + 1, + 2), + # Missing where_clause_template in TableFieldsQueryConfig. + ( + TableFieldsQueryConfig( + table_id="table", + where_clause_template="", + where_clause_fields=["id"]), + 2, + 10), + # Invalid CustomQueryConfig with None query_fn. + (CustomQueryConfig(query_fn=None), 2, 10), # type: ignore[arg-type] + # Missing table_id in TableFunctionQueryConfig. + ( + TableFunctionQueryConfig( + table_id="", + where_clause_template="id='{}'", + where_clause_value_fn=where_clause_value_fn), + 2, + 10), ]) - def test_valid_params( - self, - table_id, - where_clause_template, - where_clause_fields, - where_clause_value_fn, - query_fn, - min_batch_size, - max_batch_size): + def test_invalid_query_config( + self, query_config, min_batch_size, max_batch_size): """ - TC 1: Only batch size are provided. It should raise an error. - TC 2: Either of `where_clause_template` or `query_fn` is not provided. - TC 3: Both `where_clause_fields` and `where_clause_value_fn` are provided. - TC 4: Query construction details are provided along with `query_fn`. + TC 1: Empty TableFieldsQueryConfig. + + It should raise an error. + TC 2: Missing where_clause_template in TableFieldsQueryConfig. + TC 3: Invalid CustomQueryConfig with None query_fn. + TC 4: Missing table_id in TableFunctionQueryConfig. """ with self.assertRaises(ValueError): _ = CloudSQLEnrichmentHandler( @@ -59,15 +78,85 @@ def test_valid_params( database_user='', database_password='', database_id='', - table_id=table_id, - where_clause_template=where_clause_template, - where_clause_fields=where_clause_fields, - where_clause_value_fn=where_clause_value_fn, - query_fn=query_fn, + query_config=query_config, min_batch_size=min_batch_size, max_batch_size=max_batch_size, ) + def test_valid_query_configs(self): + """Test valid query configuration cases.""" + # Valid TableFieldsQueryConfig. + table_fields_config = TableFieldsQueryConfig( + table_id="my_table", + where_clause_template="id = '{}'", + where_clause_fields=["id"]) + + handler1 = CloudSQLEnrichmentHandler( + database_type_adapter=DatabaseTypeAdapter.POSTGRESQL, + database_address='localhost', + database_user='user', + database_password='password', + database_id='db', + query_config=table_fields_config, + min_batch_size=1, + max_batch_size=10) + + self.assertEqual( + handler1.query_template, "SELECT * FROM my_table WHERE id = '{}'") + + # Valid TableFunctionQueryConfig. + table_function_config = TableFunctionQueryConfig( + table_id="my_table", + where_clause_template="id = '{}'", + where_clause_value_fn=where_clause_value_fn) + + handler2 = CloudSQLEnrichmentHandler( + database_type_adapter=DatabaseTypeAdapter.POSTGRESQL, + database_address='localhost', + database_user='user', + database_password='password', + database_id='db', + query_config=table_function_config, + min_batch_size=1, + max_batch_size=10) + + self.assertEqual( + handler2.query_template, "SELECT * FROM my_table WHERE id = '{}'") + + # Valid CustomQueryConfig. + custom_config = CustomQueryConfig(query_fn=query_fn) + + handler3 = CloudSQLEnrichmentHandler( + database_type_adapter=DatabaseTypeAdapter.POSTGRESQL, + database_address='localhost', + database_user='user', + database_password='password', + database_id='db', + query_config=custom_config) + + # Verify that batching kwargs are empty for CustomQueryConfig. + self.assertEqual(handler3.batch_elements_kwargs(), {}) + + def test_custom_query_config_cache_key_error(self): + """Test get_cache_key raises NotImplementedError with CustomQueryConfig.""" + custom_config = CustomQueryConfig(query_fn=query_fn) + + handler = CloudSQLEnrichmentHandler( + database_type_adapter=DatabaseTypeAdapter.POSTGRESQL, + database_address='localhost', + database_user='user', + database_password='password', + database_id='db', + query_config=custom_config) + + # Create a dummy request + import apache_beam as beam + request = beam.Row(id=1) + + # Verify that get_cache_key raises NotImplementedError + with self.assertRaises(NotImplementedError): + handler.get_cache_key(request) + if __name__ == '__main__': unittest.main() From abb89b73f3c4c370898eecf371c20ab05545b6d0 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Wed, 28 May 2025 19:38:38 +0000 Subject: [PATCH 15/83] sdks/python: address claudevdm feedback (3) --- .../enrichment_handlers/cloudsql.py | 193 +++++++++--------- .../enrichment_handlers/cloudsql_it_test.py | 26 ++- .../enrichment_handlers/cloudsql_test.py | 64 ++++-- .../py310/base_image_requirements.txt | 1 + .../py311/base_image_requirements.txt | 1 + .../py312/base_image_requirements.txt | 1 + .../py39/base_image_requirements.txt | 1 + sdks/python/setup.py | 1 + 8 files changed, 171 insertions(+), 117 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py index c676f92ecdbd..c4491b43dacd 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py @@ -22,7 +22,6 @@ from typing import List from typing import Optional from typing import Union -from typing import cast from sqlalchemy import create_engine from sqlalchemy import text @@ -39,6 +38,10 @@ class CustomQueryConfig: """Configuration for using a custom query function.""" query_fn: QueryFn + def __post_init__(self): + if not self.query_fn: + raise ValueError("CustomQueryConfig must provide a valid query_fn") + @dataclass class TableFieldsQueryConfig: @@ -47,6 +50,18 @@ class TableFieldsQueryConfig: where_clause_template: str where_clause_fields: List[str] + def __post_init__(self): + if not self.table_id or not self.where_clause_template: + raise ValueError( + "TableFieldsQueryConfig and " + + "TableFunctionQueryConfig must provide table_id " + + "and where_clause_template") + + if not self.where_clause_fields: + raise ValueError( + "TableFieldsQueryConfig must provide non-empty " + + "where_clause_fields") + @dataclass class TableFunctionQueryConfig: @@ -55,47 +70,27 @@ class TableFunctionQueryConfig: where_clause_template: str where_clause_value_fn: ConditionValueFn - -QueryConfig = Union[CustomQueryConfig, - TableFieldsQueryConfig, - TableFunctionQueryConfig] - - -def _validate_query_config(query_config: QueryConfig): - """Validates the provided query configuration.""" - if isinstance(query_config, CustomQueryConfig): - if not query_config.query_fn: - raise ValueError("CustomQueryConfig must provide a valid query_fn") - elif isinstance(query_config, - (TableFieldsQueryConfig, TableFunctionQueryConfig)): - if not query_config.table_id or not query_config.where_clause_template: + def __post_init__(self): + if not self.table_id or not self.where_clause_template: raise ValueError( "TableFieldsQueryConfig and " + "TableFunctionQueryConfig must provide table_id " + "and where_clause_template") - is_table_fields = isinstance(query_config, TableFieldsQueryConfig) - if is_table_fields: - table_fields_config = cast(TableFieldsQueryConfig, query_config) - if not table_fields_config.where_clause_fields: - raise ValueError( - "TableFieldsQueryConfig must provide non-empty " + - "where_clause_fields") + if not self.where_clause_value_fn: + raise ValueError( + "TableFunctionQueryConfig must provide " + "where_clause_value_fn") + - is_table_function = isinstance(query_config, TableFunctionQueryConfig) - if is_table_function: - table_function_config = cast(TableFunctionQueryConfig, query_config) - if not table_function_config.where_clause_value_fn: - raise ValueError( - "TableFunctionQueryConfig must provide " + "where_clause_value_fn") - else: - raise ValueError("Invalid query_config type provided") +QueryConfig = Union[CustomQueryConfig, + TableFieldsQueryConfig, + TableFunctionQueryConfig] class DatabaseTypeAdapter(Enum): POSTGRESQL = "psycopg2" MYSQL = "pymysql" - SQLSERVER = "pymysql" + SQLSERVER = "pymssql" def to_sqlalchemy_dialect(self): """Map the adapter type to its corresponding SQLAlchemy dialect. @@ -190,7 +185,6 @@ def __init__( * Ensure that the database user has the necessary permissions to query the specified table. """ - _validate_query_config(query_config) self._database_type_adapter = database_type_adapter self._database_id = database_id self._database_user = database_user @@ -235,71 +229,80 @@ def _execute_query(self, query: str, is_batch: bool, **params): f'table exists. {e}') def __call__(self, request: Union[beam.Row, list[beam.Row]], *args, **kwargs): + """Handle requests by delegating to single or batch processing.""" if isinstance(request, list): - values, responses = [], [] - requests_map: dict[Any, Any] = {} - batch_size = len(request) - raw_query = self.query_template - - # For multiple requests in the batch, combine the WHERE clause conditions - # using 'OR' and update the query template to handle all requests. - table_query_configs = (TableFieldsQueryConfig, TableFunctionQueryConfig) - if batch_size > 1 and isinstance(self._query_config, table_query_configs): - where_clause_template_batched = ' OR '.join( - [fr'({self._query_config.where_clause_template})'] * batch_size) - raw_query = self.query_template.replace( - self._query_config.where_clause_template, - where_clause_template_batched) - - # Extract where_clause_fields values and map the generated request key to - # the original request object. - for req in request: - request_dict = req._asdict() - try: - if isinstance(self._query_config, TableFunctionQueryConfig): - current_values = self._query_config.where_clause_value_fn(req) - elif isinstance(self._query_config, TableFieldsQueryConfig): - current_values = [ - request_dict[field] - for field in self._query_config.where_clause_fields - ] - except KeyError as e: - raise KeyError( - "Make sure the values passed in `where_clause_fields` are the " - "keys in the input `beam.Row`." + str(e)) - values.extend(current_values) - requests_map[self.create_row_key(req)] = req - - # Formulate the query, execute it, and return a list of original requests - # paired with their responses. - query = raw_query.format(*values) - responses_dict = self._execute_query(query, is_batch=True) - for response in responses_dict: - response_row = beam.Row(**response) - response_key = self.create_row_key(response_row) - if response_key in requests_map: - responses.append((requests_map[response_key], response_row)) - return responses + return self._process_batch_request(request) else: - request_dict = request._asdict() - if isinstance(self._query_config, CustomQueryConfig): - query = self._query_config.query_fn(request) - else: - try: - if isinstance(self._query_config, TableFunctionQueryConfig): - values = self._query_config.where_clause_value_fn(request) - elif isinstance(self._query_config, TableFieldsQueryConfig): - values = [ - request_dict[field] - for field in self._query_config.where_clause_fields - ] - except KeyError as e: - raise KeyError( - "Make sure the values passed in `where_clause_fields` are the " - "keys in the input `beam.Row`." + str(e)) - query = self.query_template.format(*values) - response_dict = self._execute_query(query, is_batch=False) - return request, beam.Row(**response_dict) + return self._process_single_request(request) + + def _process_batch_request(self, requests: list[beam.Row]): + """Process batch requests and match responses to original requests.""" + values, responses = [], [] + requests_map: dict[Any, Any] = {} + batch_size = len(requests) + raw_query = self.query_template + + # For multiple requests in the batch, combine the WHERE clause conditions + # using 'OR' and update the query template to handle all requests. + table_query_configs = (TableFieldsQueryConfig, TableFunctionQueryConfig) + if batch_size > 1 and isinstance(self._query_config, table_query_configs): + where_clause_template_batched = ' OR '.join( + [fr'({self._query_config.where_clause_template})'] * batch_size) + raw_query = self.query_template.replace( + self._query_config.where_clause_template, + where_clause_template_batched) + + # Extract where_clause_fields values and map the generated request key to + # the original request object. + for req in requests: + request_dict = req._asdict() + try: + if isinstance(self._query_config, TableFunctionQueryConfig): + current_values = self._query_config.where_clause_value_fn(req) + elif isinstance(self._query_config, TableFieldsQueryConfig): + current_values = [ + request_dict[field] + for field in self._query_config.where_clause_fields + ] + except KeyError as e: + raise KeyError( + "Make sure the values passed in `where_clause_fields` are " + " thekeys in the input `beam.Row`." + str(e)) + values.extend(current_values) + requests_map[self.create_row_key(req)] = req + + # Formulate the query, execute it, and return a list of original requests + # paired with their responses. + query = raw_query.format(*values) + responses_dict = self._execute_query(query, is_batch=True) + for response in responses_dict: + response_row = beam.Row(**response) + response_key = self.create_row_key(response_row) + if response_key in requests_map: + responses.append((requests_map[response_key], response_row)) + return responses + + def _process_single_request(self, request: beam.Row): + """Process a single request and return with its response.""" + request_dict = request._asdict() + if isinstance(self._query_config, CustomQueryConfig): + query = self._query_config.query_fn(request) + else: + try: + if isinstance(self._query_config, TableFunctionQueryConfig): + values = self._query_config.where_clause_value_fn(request) + elif isinstance(self._query_config, TableFieldsQueryConfig): + values = [ + request_dict[field] + for field in self._query_config.where_clause_fields + ] + except KeyError as e: + raise KeyError( + "Make sure the values passed in `where_clause_fields` are " + "the keys in the input `beam.Row`." + str(e)) + query = self.query_template.format(*values) + response_dict = self._execute_query(query, is_batch=False) + return request, beam.Row(**response_dict) def create_row_key(self, row: beam.Row): if isinstance(self._query_config, TableFunctionQueryConfig): diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py index e989be7079ba..6c12e5888c05 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py @@ -88,25 +88,39 @@ def start_sql_db_container( for i in range(sql_client_retries): try: if database_type == DatabaseTypeAdapter.POSTGRESQL: - sql_db_container = PostgresContainer(image="postgres:16") + user, password, db_id = "test", "test", "test" + sql_db_container = PostgresContainer( + image="postgres:16", + user=user, + password=password, + dbname=db_id, + driver=database_type.value) sql_db_container.start() host = sql_db_container.get_container_host_ip() port = sql_db_container.get_exposed_port(5432) - user, password, db_id = "test", "test", "test" elif database_type == DatabaseTypeAdapter.MYSQL: - sql_db_container = MySqlContainer(image="mysql:8.0") + user, password, db_id = "test", "test", "test" + sql_db_container = MySqlContainer( + image="mysql:8.0", + MYSQL_USER=user, + MYSQL_ROOT_PASSWORD=password, + MYSQL_PASSWORD=password, + MYSQL_DATABASE=db_id) sql_db_container.start() host = sql_db_container.get_container_host_ip() port = sql_db_container.get_exposed_port(3306) - user, password, db_id = "test", "test", "test" elif database_type == DatabaseTypeAdapter.SQLSERVER: - sql_db_container = SqlServerContainer() + user, password, db_id = "SA", "A_Str0ng_Required_Password", "tempdb" + sql_db_container = SqlServerContainer( + image="mcr.microsoft.com/mssql/server:2022-latest", + user=user, + password=password, + dbname=db_id) sql_db_container.start() host = sql_db_container.get_container_host_ip() port = sql_db_container.get_exposed_port(1433) - user, password, db_id = "sa", "A_Str0ng_Required_Password", "tempdb" else: raise ValueError(f"Unsupported database type: {database_type}") diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_test.py index f3da541cd89a..30aeca4d04f6 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_test.py @@ -38,40 +38,70 @@ class TestCloudSQLEnrichment(unittest.TestCase): @parameterized.expand([ # Empty TableFieldsQueryConfig. ( - TableFieldsQueryConfig( + lambda: TableFieldsQueryConfig( table_id="", where_clause_template="", where_clause_fields=[]), 1, - 2), + 2, + "must provide table_id and where_clause_template" + ), # Missing where_clause_template in TableFieldsQueryConfig. ( - TableFieldsQueryConfig( + lambda: TableFieldsQueryConfig( table_id="table", where_clause_template="", where_clause_fields=["id"]), 2, - 10), + 10, + "must provide table_id and where_clause_template" + ), # Invalid CustomQueryConfig with None query_fn. - (CustomQueryConfig(query_fn=None), 2, 10), # type: ignore[arg-type] + ( + lambda: CustomQueryConfig(query_fn=None), # type: ignore[arg-type] + 2, + 10, + "must provide a valid query_fn" + ), # Missing table_id in TableFunctionQueryConfig. ( - TableFunctionQueryConfig( + lambda: TableFunctionQueryConfig( table_id="", where_clause_template="id='{}'", where_clause_value_fn=where_clause_value_fn), 2, - 10), + 10, + "must provide table_id and where_clause_template" + ), + # Missing where_clause_fields in TableFieldsQueryConfig. + ( + lambda: TableFieldsQueryConfig( + table_id="table", + where_clause_template="id = '{}'", + where_clause_fields=[]), + 1, + 10, + "must provide non-empty where_clause_fields" + ), + # Missing where_clause_value_fn in TableFunctionQueryConfig. + ( + lambda: TableFunctionQueryConfig( + table_id="table", + where_clause_template="id = '{}'", + where_clause_value_fn=None), # type: ignore[arg-type] + 1, + 10, + "must provide where_clause_value_fn" + ), ]) def test_invalid_query_config( - self, query_config, min_batch_size, max_batch_size): - """ - TC 1: Empty TableFieldsQueryConfig. - - It should raise an error. - TC 2: Missing where_clause_template in TableFieldsQueryConfig. - TC 3: Invalid CustomQueryConfig with None query_fn. - TC 4: Missing table_id in TableFunctionQueryConfig. + self, create_config, min_batch_size, max_batch_size, expected_error_msg): + """Test that validation errors are raised for invalid query configs. + + The test verifies both that the appropriate ValueError is raised and that + the error message contains the expected text. """ - with self.assertRaises(ValueError): + with self.assertRaises(ValueError) as context: + # Call the lambda to create the config. + query_config = create_config() _ = CloudSQLEnrichmentHandler( database_type_adapter=DatabaseTypeAdapter.POSTGRESQL, database_address='', @@ -82,6 +112,8 @@ def test_invalid_query_config( min_batch_size=min_batch_size, max_batch_size=max_batch_size, ) + # Verify the error message contains the expected text. + self.assertIn(expected_error_msg, str(context.exception)) def test_valid_query_configs(self): """Test valid query configuration cases.""" diff --git a/sdks/python/container/py310/base_image_requirements.txt b/sdks/python/container/py310/base_image_requirements.txt index d8c84479b50f..08ec0b990773 100644 --- a/sdks/python/container/py310/base_image_requirements.txt +++ b/sdks/python/container/py310/base_image_requirements.txt @@ -119,6 +119,7 @@ pluggy==1.5.0 proto-plus==1.26.1 protobuf==5.29.4 psycopg2-binary==2.9.9 +pymssql==2.3.4 pyarrow==16.1.0 pyarrow-hotfix==0.6 pyasn1==0.6.1 diff --git a/sdks/python/container/py311/base_image_requirements.txt b/sdks/python/container/py311/base_image_requirements.txt index dc0c3f2b95a6..6bc29d658707 100644 --- a/sdks/python/container/py311/base_image_requirements.txt +++ b/sdks/python/container/py311/base_image_requirements.txt @@ -117,6 +117,7 @@ pluggy==1.5.0 proto-plus==1.26.1 protobuf==5.29.4 psycopg2-binary==2.9.9 +pymssql==2.3.4 pyarrow==16.1.0 pyarrow-hotfix==0.6 pyasn1==0.6.1 diff --git a/sdks/python/container/py312/base_image_requirements.txt b/sdks/python/container/py312/base_image_requirements.txt index a34f4ccef489..ef04011e46e6 100644 --- a/sdks/python/container/py312/base_image_requirements.txt +++ b/sdks/python/container/py312/base_image_requirements.txt @@ -116,6 +116,7 @@ pluggy==1.5.0 proto-plus==1.26.1 protobuf==5.29.4 psycopg2-binary==2.9.9 +pymssql==2.3.4 pyarrow==16.1.0 pyarrow-hotfix==0.6 pyasn1==0.6.1 diff --git a/sdks/python/container/py39/base_image_requirements.txt b/sdks/python/container/py39/base_image_requirements.txt index 6be1fdd3b0d4..7cb9eae1b12f 100644 --- a/sdks/python/container/py39/base_image_requirements.txt +++ b/sdks/python/container/py39/base_image_requirements.txt @@ -119,6 +119,7 @@ pluggy==1.5.0 proto-plus==1.26.1 protobuf==5.29.4 psycopg2-binary==2.9.9 +pymssql==2.3.4 pyarrow==16.1.0 pyarrow-hotfix==0.6 pyasn1==0.6.1 diff --git a/sdks/python/setup.py b/sdks/python/setup.py index e1756176093b..1afc48cb8967 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -418,6 +418,7 @@ def get_portability_package_data(): 'sqlalchemy>=1.3,<3.0', 'psycopg2-binary>=2.8.5,<3.0.0,!=2.9.10', 'testcontainers[mysql]>=3.0.3,<4.0.0', + 'pymssql>=2.3.4,<3.0.0', 'cryptography>=41.0.2', 'hypothesis>5.0.0,<7.0.0', 'virtualenv-clone>=0.5,<1.0', From bc84430bdc14e102fb1a23222b58236d14024eb0 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Fri, 6 Jun 2025 14:55:04 +0000 Subject: [PATCH 16/83] multi: fix pymssql mac os deps --- sdks/python/scripts/install_macos_deps.sh | 45 +++++++++++++++++++++++ sdks/python/setup.py | 2 +- sdks/python/tox.ini | 1 + 3 files changed, 47 insertions(+), 1 deletion(-) create mode 100755 sdks/python/scripts/install_macos_deps.sh diff --git a/sdks/python/scripts/install_macos_deps.sh b/sdks/python/scripts/install_macos_deps.sh new file mode 100755 index 000000000000..a8210048fcc2 --- /dev/null +++ b/sdks/python/scripts/install_macos_deps.sh @@ -0,0 +1,45 @@ +#!/bin/bash +# +# 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. +# + +set -euo pipefail + +# Create temporary directory. +TMPDIR=$(mktemp -d -t install_macos_deps.XXXXXX) +cleanup() { + rm -rf "$TMPDIR" +} +trap cleanup EXIT + +echo "Using temporary directory: $TMPDIR" +cd "$TMPDIR" + +# Download and extract FreeTDS. +curl -LO https://www.freetds.org/files/stable/freetds-1.5.2.tar.gz +tar -xzf freetds-1.5.2.tar.gz +cd freetds-1.5.2 + +# Configure, build and install FreeTDS. +./configure --prefix="$HOME/freetds" --with-tdsver=7.4 +make +make install + +# Set environment variables for pymssql installation. +export CFLAGS="-I$HOME/freetds/include" +export LDFLAGS="-L$HOME/freetds/lib" + +echo "FreeTDS installed to $HOME/freetds" diff --git a/sdks/python/setup.py b/sdks/python/setup.py index 656983042a96..76b50d71337a 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -431,7 +431,7 @@ def get_portability_package_data(): 'psycopg2-binary>=2.8.5,<2.9.10; python_version <= "3.9"', 'psycopg2-binary>=2.8.5,<3.0; python_version >= "3.10"', 'testcontainers[mysql,kafka]>=3.0.3,<4.0.0', - 'pymssql>=2.3.4,<3.0.0', + 'pymssql>=2.3.4,<3.0.0; python_version >= "3.8"', 'cryptography>=41.0.2', 'hypothesis>5.0.0,<7.0.0', 'virtualenv-clone>=0.5,<1.0', diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini index 5131769509d9..741a5d74b215 100644 --- a/sdks/python/tox.ini +++ b/sdks/python/tox.ini @@ -84,6 +84,7 @@ commands_pre = python --version pip --version # pip check + bash {toxinidir}/scripts/install_macos_deps.sh bash {toxinidir}/scripts/run_tox_cleanup.sh commands = python apache_beam/examples/complete/autocomplete_test.py From 7a56f0e73b3196bfcc7a17d50c494620a815daac Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Fri, 6 Jun 2025 16:36:29 +0000 Subject: [PATCH 17/83] multi: use pytds sql server database adapter In this commit, we use pytds microsoft sqlserver adapter instead of pymssql for one main reason it supports cross-platform compatibility epsecially with macOS so we don't need to install FreeTDS C dependencies. --- .../enrichment_handlers/cloudsql.py | 2 +- .../enrichment_handlers/cloudsql_it_test.py | 3 +- .../py310/base_image_requirements.txt | 1 - .../py311/base_image_requirements.txt | 1 - .../py312/base_image_requirements.txt | 1 - .../py39/base_image_requirements.txt | 1 - sdks/python/scripts/install_macos_deps.sh | 45 ------------------- sdks/python/setup.py | 1 - sdks/python/tox.ini | 1 - 9 files changed, 3 insertions(+), 53 deletions(-) delete mode 100755 sdks/python/scripts/install_macos_deps.sh diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py index c4491b43dacd..39ef2eb50fe7 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py @@ -90,7 +90,7 @@ def __post_init__(self): class DatabaseTypeAdapter(Enum): POSTGRESQL = "psycopg2" MYSQL = "pymysql" - SQLSERVER = "pymssql" + SQLSERVER = "pytds" def to_sqlalchemy_dialect(self): """Map the adapter type to its corresponding SQLAlchemy dialect. diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py index 6c12e5888c05..f44819c115af 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py @@ -117,7 +117,8 @@ def start_sql_db_container( image="mcr.microsoft.com/mssql/server:2022-latest", user=user, password=password, - dbname=db_id) + dbname=db_id, + dialect=database_type.to_sqlalchemy_dialect()) sql_db_container.start() host = sql_db_container.get_container_host_ip() port = sql_db_container.get_exposed_port(1433) diff --git a/sdks/python/container/py310/base_image_requirements.txt b/sdks/python/container/py310/base_image_requirements.txt index 3699d37e32e2..71dbfb4dd9cb 100644 --- a/sdks/python/container/py310/base_image_requirements.txt +++ b/sdks/python/container/py310/base_image_requirements.txt @@ -124,7 +124,6 @@ protobuf==5.29.4 psycopg2-binary==2.9.10 pyarrow==18.1.0 pyarrow-hotfix==0.7 -pymssql==2.3.4 pyasn1==0.6.1 pyasn1_modules==0.4.2 pycparser==2.22 diff --git a/sdks/python/container/py311/base_image_requirements.txt b/sdks/python/container/py311/base_image_requirements.txt index 3f08c4816855..4a818db73073 100644 --- a/sdks/python/container/py311/base_image_requirements.txt +++ b/sdks/python/container/py311/base_image_requirements.txt @@ -122,7 +122,6 @@ protobuf==5.29.4 psycopg2-binary==2.9.10 pyarrow==18.1.0 pyarrow-hotfix==0.7 -pymssql==2.3.4 pyasn1==0.6.1 pyasn1_modules==0.4.2 pycparser==2.22 diff --git a/sdks/python/container/py312/base_image_requirements.txt b/sdks/python/container/py312/base_image_requirements.txt index d146010af0c4..44a3e8d21046 100644 --- a/sdks/python/container/py312/base_image_requirements.txt +++ b/sdks/python/container/py312/base_image_requirements.txt @@ -121,7 +121,6 @@ protobuf==5.29.4 psycopg2-binary==2.9.10 pyarrow==18.1.0 pyarrow-hotfix==0.7 -pymssql==2.3.4 pyasn1==0.6.1 pyasn1_modules==0.4.2 pycparser==2.22 diff --git a/sdks/python/container/py39/base_image_requirements.txt b/sdks/python/container/py39/base_image_requirements.txt index 1fab7e2d2e28..6591f108a99e 100644 --- a/sdks/python/container/py39/base_image_requirements.txt +++ b/sdks/python/container/py39/base_image_requirements.txt @@ -124,7 +124,6 @@ protobuf==5.29.4 psycopg2-binary==2.9.9 pyarrow==18.1.0 pyarrow-hotfix==0.7 -pymssql==2.3.4 pyasn1==0.6.1 pyasn1_modules==0.4.2 pycparser==2.22 diff --git a/sdks/python/scripts/install_macos_deps.sh b/sdks/python/scripts/install_macos_deps.sh deleted file mode 100755 index a8210048fcc2..000000000000 --- a/sdks/python/scripts/install_macos_deps.sh +++ /dev/null @@ -1,45 +0,0 @@ -#!/bin/bash -# -# 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. -# - -set -euo pipefail - -# Create temporary directory. -TMPDIR=$(mktemp -d -t install_macos_deps.XXXXXX) -cleanup() { - rm -rf "$TMPDIR" -} -trap cleanup EXIT - -echo "Using temporary directory: $TMPDIR" -cd "$TMPDIR" - -# Download and extract FreeTDS. -curl -LO https://www.freetds.org/files/stable/freetds-1.5.2.tar.gz -tar -xzf freetds-1.5.2.tar.gz -cd freetds-1.5.2 - -# Configure, build and install FreeTDS. -./configure --prefix="$HOME/freetds" --with-tdsver=7.4 -make -make install - -# Set environment variables for pymssql installation. -export CFLAGS="-I$HOME/freetds/include" -export LDFLAGS="-L$HOME/freetds/lib" - -echo "FreeTDS installed to $HOME/freetds" diff --git a/sdks/python/setup.py b/sdks/python/setup.py index 76b50d71337a..a98eaab33361 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -431,7 +431,6 @@ def get_portability_package_data(): 'psycopg2-binary>=2.8.5,<2.9.10; python_version <= "3.9"', 'psycopg2-binary>=2.8.5,<3.0; python_version >= "3.10"', 'testcontainers[mysql,kafka]>=3.0.3,<4.0.0', - 'pymssql>=2.3.4,<3.0.0; python_version >= "3.8"', 'cryptography>=41.0.2', 'hypothesis>5.0.0,<7.0.0', 'virtualenv-clone>=0.5,<1.0', diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini index 741a5d74b215..5131769509d9 100644 --- a/sdks/python/tox.ini +++ b/sdks/python/tox.ini @@ -84,7 +84,6 @@ commands_pre = python --version pip --version # pip check - bash {toxinidir}/scripts/install_macos_deps.sh bash {toxinidir}/scripts/run_tox_cleanup.sh commands = python apache_beam/examples/complete/autocomplete_test.py From cc7c7f095c5124f66b22e5fbb1c7ec63033c22cc Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Fri, 6 Jun 2025 18:10:28 +0000 Subject: [PATCH 18/83] sdks/python: use `VARCHAR` datatype --- .../transforms/enrichment_handlers/cloudsql_it_test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py index f44819c115af..8c6c1f9feb37 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py @@ -37,7 +37,7 @@ from testcontainers.mssql import SqlServerContainer from testcontainers.redis import RedisContainer from sqlalchemy import ( - create_engine, MetaData, Table, Column, Integer, String, Engine) + create_engine, MetaData, Table, Column, Integer, VARCHAR, Engine) from apache_beam.transforms.enrichment import Enrichment from apache_beam.transforms.enrichment_handlers.cloudsql import ( CloudSQLEnrichmentHandler, @@ -245,7 +245,7 @@ def get_columns(cls): """Returns fresh column objects each time it's called.""" return [ Column("id", Integer, primary_key=True), - Column("name", String(255), nullable=False), + Column("name", VARCHAR(255), nullable=False), Column("quantity", Integer, nullable=False), Column("distribution_center_id", Integer, nullable=False), ] From 7a7436da133000f86bd097c224f615f070dff99a Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Thu, 12 Jun 2025 11:59:55 +0000 Subject: [PATCH 19/83] sdks/python: address claudevdm feedback (4) --- .../transforms/elementwise/enrichment.py | 22 +- .../transforms/elementwise/enrichment_test.py | 51 ++++- .../enrichment_handlers/cloudsql.py | 196 +++++++++++++---- .../enrichment_handlers/cloudsql_it_test.py | 205 ++++++++++++------ .../enrichment_handlers/cloudsql_test.py | 62 +++--- .../py310/base_image_requirements.txt | 93 ++++---- .../py311/base_image_requirements.txt | 93 ++++---- .../py312/base_image_requirements.txt | 95 ++++---- .../py313/base_image_requirements.txt | 93 ++++---- .../py39/base_image_requirements.txt | 89 ++++---- sdks/python/setup.py | 8 +- 11 files changed, 655 insertions(+), 352 deletions(-) diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py index 0190b12b700d..edfccda75350 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py @@ -56,11 +56,15 @@ def enrichment_with_cloudsql(): import apache_beam as beam from apache_beam.transforms.enrichment import Enrichment from apache_beam.transforms.enrichment_handlers.cloudsql import ( - CloudSQLEnrichmentHandler, DatabaseTypeAdapter, TableFieldsQueryConfig) + CloudSQLEnrichmentHandler, + DatabaseTypeAdapter, + TableFieldsQueryConfig, + ExternalSQLDBConnectionConfig) import os database_type_adapter = DatabaseTypeAdapter[os.environ.get("SQL_DB_TYPE")] - database_address = os.environ.get("SQL_DB_ADDRESS") + database_host = os.environ.get("SQL_DB_HOST") + database_port = int(os.environ.get("SQL_DB_PORT")) database_user = os.environ.get("SQL_DB_USER") database_password = os.environ.get("SQL_DB_PASSWORD") database_id = os.environ.get("SQL_DB_ID") @@ -74,17 +78,21 @@ def enrichment_with_cloudsql(): beam.Row(product_id=3, name='C'), ] + connection_config = ExternalSQLDBConnectionConfig( + db_adapter=database_type_adapter, + host=database_host, + port=database_port, + user=database_user, + password=database_password, + db_id=database_id) + query_config = TableFieldsQueryConfig( table_id=table_id, where_clause_template=where_clause_template, where_clause_fields=where_clause_fields) cloudsql_handler = CloudSQLEnrichmentHandler( - database_type_adapter=database_type_adapter, - database_address=database_address, - database_user=database_user, - database_password=database_password, - database_id=database_id, + connection_config=connection_config, table_id=table_id, query_config=query_config) with beam.Pipeline() as p: diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py index 426f96d58b8e..36b46cb5a74e 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py @@ -21,13 +21,14 @@ import os import unittest from io import StringIO +from typing import Tuple import mock import pytest # pylint: disable=unused-import try: - from sqlalchemy import Column, Integer, String, Engine + from sqlalchemy import Column, Integer, String, Engine, MetaData from apache_beam.examples.snippets.transforms.elementwise.enrichment import ( enrichment_with_bigtable, enrichment_with_vertex_ai_legacy) from apache_beam.examples.snippets.transforms.elementwise.enrichment import ( @@ -35,7 +36,10 @@ from apache_beam.transforms.enrichment_handlers.cloudsql import ( DatabaseTypeAdapter) from apache_beam.transforms.enrichment_handlers.cloudsql_it_test import ( - CloudSQLEnrichmentTestHelper, SQLDBContainerInfo) + CloudSQLEnrichmentTestHelper, + SQLDBContainerInfo, + ExternalSQLDBConnectionConfig, + SQLClientConnectionHandler) from apache_beam.io.requestresponse import RequestResponseIO except ImportError: raise unittest.SkipTest('RequestResponseIO dependencies are not installed') @@ -89,7 +93,7 @@ def test_enrichment_with_bigtable(self, mock_stdout): def test_enrichment_with_cloudsql(self, mock_stdout): db, engine = None, None try: - db, engine = self.pre_cloudsql_enrichment_test() + db, handler, metadata, engine = self.pre_cloudsql_enrichment_test() enrichment_with_cloudsql() output = mock_stdout.getvalue().splitlines() expected = validate_enrichment_with_cloudsql() @@ -98,7 +102,11 @@ def test_enrichment_with_cloudsql(self, mock_stdout): self.fail(f"Test failed with unexpected error: {e}") finally: if db and engine: - self.post_cloudsql_enrichment_test(db, engine) + self.post_cloudsql_enrichment_test( + db=db, + sql_client_connection_handler=handler, + metadata=metadata, + engine=engine) def test_enrichment_with_vertex_ai(self, mock_stdout): enrichment_with_vertex_ai() @@ -134,27 +142,48 @@ def pre_cloudsql_enrichment_test(self): "product_id": 3, "name": "C", 'quantity': 10, 'region_id': 4 }, ] + metadata = MetaData() db_adapter = DatabaseTypeAdapter.POSTGRESQL db = CloudSQLEnrichmentTestHelper.start_sql_db_container(db_adapter) os.environ['SQL_DB_TYPE'] = db.adapter.name - os.environ['SQL_DB_ADDRESS'] = db.address + os.environ['SQL_DB_HOST'] = db.host + os.environ['SQL_DB_PORT'] = str(db.port) os.environ['SQL_DB_USER'] = db.user os.environ['SQL_DB_PASSWORD'] = db.password os.environ['SQL_DB_ID'] = db.id os.environ['SQL_DB_URL'] = db.url - engine = CloudSQLEnrichmentTestHelper.create_table( - db_url=os.environ.get("SQL_DB_URL"), + connection_config = ExternalSQLDBConnectionConfig( + db_adapter=db_adapter, + host=db.host, + port=db.port, + user=db.user, + password=db.password, + db_id=db.id) + handler = CloudSQLEnrichmentTestHelper.create_table( table_id=table_id, + connection_config=connection_config, columns=columns, - table_data=table_data) - return db, engine + table_data=table_data, + metadata=metadata) + + handler: Tuple[SQLClientConnectionHandler, Engine] + + sql_client_connection_handler, engine = handler + return db, sql_client_connection_handler, metadata, engine def post_cloudsql_enrichment_test( - self, db: SQLDBContainerInfo, engine: Engine): + self, + db: SQLDBContainerInfo, + sql_client_connection_handler: SQLClientConnectionHandler, + metadata: MetaData, + engine: Engine): + metadata.drop_all(engine) + sql_client_connection_handler.connection_closer() engine.dispose(close=True) CloudSQLEnrichmentTestHelper.stop_sql_db_container(db) os.environ.pop('SQL_DB_TYPE', None) - os.environ.pop('SQL_DB_ADDRESS', None) + os.environ.pop('SQL_DB_HOST', None) + os.environ.pop('SQL_DB_PORT', None) os.environ.pop('SQL_DB_USER', None) os.environ.pop('SQL_DB_PASSWORD', None) os.environ.pop('SQL_DB_ID', None) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py index 39ef2eb50fe7..6acb437cb782 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py @@ -14,6 +14,7 @@ # See the License for the specific language governing permissions and # limitations under the License. # +from abc import abstractmethod, ABC from collections.abc import Callable from collections.abc import Mapping from dataclasses import dataclass @@ -23,8 +24,15 @@ from typing import Optional from typing import Union +import pymysql +import pg8000 +import pytds from sqlalchemy import create_engine from sqlalchemy import text +from sqlalchemy.engine import Connection as DBAPIConnection +from google.cloud.sql.connector.enums import RefreshStrategy +from google.cloud.sql.connector import Connector as CloudSQLConnector +from google.cloud.sql.connector import IPTypes import apache_beam as beam from apache_beam.transforms.enrichment import EnrichmentSourceHandler @@ -82,13 +90,8 @@ def __post_init__(self): "TableFunctionQueryConfig must provide " + "where_clause_value_fn") -QueryConfig = Union[CustomQueryConfig, - TableFieldsQueryConfig, - TableFunctionQueryConfig] - - class DatabaseTypeAdapter(Enum): - POSTGRESQL = "psycopg2" + POSTGRESQL = "pg8000" MYSQL = "pymysql" SQLSERVER = "pytds" @@ -105,7 +108,141 @@ def to_sqlalchemy_dialect(self): elif self == DatabaseTypeAdapter.SQLSERVER: return f"mssql+{self.value}" else: - raise ValueError(f"Unsupported adapter type: {self.name}") + raise ValueError(f"Unsupported database adapter type: {self.name}") + + +@dataclass +class SQLClientConnectionHandler: + connector: Callable[[], DBAPIConnection] + connection_closer: Callable[[], None] + + +class ConnectionConfig(ABC): + @abstractmethod + def get_connector_handler(self) -> SQLClientConnectionHandler: + pass + + @abstractmethod + def get_db_url(self) -> str: + pass + + +class CloudSQLConnectionConfig(ConnectionConfig): + """Connects to Google Cloud SQL using Cloud SQL Python Connector.""" + SUPPORTED_ADAPTERS = { + DatabaseTypeAdapter.POSTGRESQL, + DatabaseTypeAdapter.MYSQL, + DatabaseTypeAdapter.SQLSERVER, + } + + def __init__( + self, + db_adapter: DatabaseTypeAdapter, + instance_connection_name: str, + user: str, + db_id: str, + enable_iam_auth: bool = True, + password: Optional[str] = None, # fallback if IAM not used + ip_type: IPTypes = IPTypes.PUBLIC, + refresh_strategy: RefreshStrategy = RefreshStrategy.LAZY, + *, + connector_kwargs: dict = {}, + connect_kwargs: dict = {}, + ): + self._validate_metadata(db_adapter=db_adapter) + self._db_adapter = db_adapter + self._instance_connection_name = instance_connection_name + self._user = user + self._db_id = db_id + self._enable_iam_auth = enable_iam_auth + self._password = password + self._ip_type = ip_type + self._refresh_strategy = refresh_strategy + self.connector_kwargs = connector_kwargs + self.connect_kwargs = connect_kwargs + + def get_connector_handler(self) -> SQLClientConnectionHandler: + cloudsql_client = CloudSQLConnector( + ip_type=self._ip_type, + refresh_strategy=self._refresh_strategy, + **self.connector_kwargs) + + cloudsql_connector = lambda: cloudsql_client.connect( + instance_connection_string=self._instance_connection_name, driver=self. + _db_adapter.value, user=self._user, db=self._db_id, enable_iam_auth=self + ._enable_iam_auth, password=self.password, **self.connect_kwargs) + + connection_closer = lambda: cloudsql_client.close() + + return SQLClientConnectionHandler( + connector=cloudsql_connector, connection_closer=connection_closer) + + def get_db_url(self) -> str: + return self._db_adapter.to_sqlalchemy_dialect() + "://" + + def _validate_metadata(self, db_adapter: DatabaseTypeAdapter): + if db_adapter not in self.SUPPORTED_ADAPTERS: + raise ValueError( + f"Unsupported DB adapter for CloudSQLConnectionConfig: {db_adapter}. " + f"Supported: {[a.name for a in self.SUPPORTED_ADAPTERS]}") + + @property + def password(self): + return self._password if not self._enable_iam_auth else None + + +class ExternalSQLDBConnectionConfig(ConnectionConfig): + """Connects to External SQL databases (PostgreSQL, MySQL, etc.) over TCP.""" + def __init__( + self, + db_adapter: DatabaseTypeAdapter, + host: str, + user: str, + password: str, + db_id: str, + port: int, + **kwargs): + self._db_adapter = db_adapter + self._host = host + self._user = user + self._password = password + self._db_id = db_id + self._port = port + self.kwargs = kwargs + + def get_connector_handler( + self, + ) -> SQLClientConnectionHandler: + if self._db_adapter == DatabaseTypeAdapter.POSTGRESQL: + # It is automatically closed upstream by sqlalchemy. + connector = lambda: pg8000.connect( + host=self._host, user=self._user, password=self._password, database= + self._db_id, port=self._port, **self.kwargs) + connection_closer = lambda: None + elif self._db_adapter == DatabaseTypeAdapter.MYSQL: + # It is automatically closed upstream by sqlalchemy. + connector = lambda: pymysql.connect( + host=self._host, user=self._user, password=self._password, database= + self._db_id, port=self._port, **self.kwargs) + connection_closer = lambda: None + elif self._db_adapter == DatabaseTypeAdapter.SQLSERVER: + # It is automatically closed upstream by sqlalchemy. + connector = lambda: pytds.connect( + server=self._host, database=self._db_id, user=self._user, password= + self._password, port=self._port, **self.kwargs) + connection_closer = lambda: None + else: + raise ValueError(f"Unsupported DB adapter: {self._db_adapter}") + + return SQLClientConnectionHandler(connector, connection_closer) + + def get_db_url(self) -> str: + return self._db_adapter.to_sqlalchemy_dialect() + "://" + + +QueryConfig = Union[CustomQueryConfig, + TableFieldsQueryConfig, + TableFunctionQueryConfig] class CloudSQLEnrichmentHandler(EnrichmentSourceHandler[beam.Row, beam.Row]): @@ -132,11 +269,7 @@ class CloudSQLEnrichmentHandler(EnrichmentSourceHandler[beam.Row, beam.Row]): """ def __init__( self, - database_type_adapter: DatabaseTypeAdapter, - database_address: str, - database_user: str, - database_password: str, - database_id: str, + connection_config: ConnectionConfig, *, query_config: QueryConfig, column_names: Optional[list[str]] = None, @@ -147,24 +280,17 @@ def __init__( """ Example Usage: handler = CloudSQLEnrichmentHandler( - database_type_adapter=adapter, - database_address='127.0.0.1:5432', - database_user='user', - database_password='password', - database_id='my_database', + connection_config=CloudSQLConnectionConfig(...), query_config=TableFieldsQueryConfig('my_table',"id = '{}'",['id']), min_batch_size=2, max_batch_size=100) Args: - database_type_adapter: Adapter to handle specific database type operations - (e.g., MySQL, PostgreSQL). - database_address (str): Address or hostname of the Cloud SQL database, in - the form `:`. The port is optional if the database uses - the default port. - database_user (str): Username for accessing the database. - database_password (str): Password for accessing the database. - database_id (str): Identifier for the database to query. + connection_config (ConnectionConfig): Configuration for connecting to + the SQL database. Must be an instance of a subclass of + `ConnectionConfig`, such as `CloudSQLConnectionConfig` or + `ExternalSQLDBConnectionConfig`. This determines how the handler + connects to the target SQL database. query_config: Configuration for database queries. Must be one of: * CustomQueryConfig: For providing a custom query function * TableFieldsQueryConfig: specifies table, where clause, and field names @@ -185,11 +311,7 @@ def __init__( * Ensure that the database user has the necessary permissions to query the specified table. """ - self._database_type_adapter = database_type_adapter - self._database_id = database_id - self._database_user = database_user - self._database_password = database_password - self._database_address = database_address + self._connection_config = connection_config self._query_config = query_config self._column_names = ",".join(column_names) if column_names else "*" self.kwargs = kwargs @@ -204,17 +326,12 @@ def __init__( self._batching_kwargs['max_batch_size'] = max_batch_size def __enter__(self): - db_url = self._get_db_url() - self._engine = create_engine(db_url) + self._sql_client_handler = self._connection_config.get_connector_handler() + self._engine = create_engine( + url=self._connection_config.get_db_url(), + creator=self._sql_client_handler.connector) self._connection = self._engine.connect() - def _get_db_url(self) -> str: - dialect = self._database_type_adapter.to_sqlalchemy_dialect() - url = ( - f"{dialect}://{self._database_user}:{self._database_password}" - f"@{self._database_address}/{self._database_id}") - return url - def _execute_query(self, query: str, is_batch: bool, **params): try: result = self._connection.execute(text(query), **params) @@ -318,6 +435,7 @@ def create_row_key(self, row: beam.Row): def __exit__(self, exc_type, exc_val, exc_tb): self._connection.close() + self._sql_client_handler.connection_closer() self._engine.dispose(close=True) self._engine, self._connection = None, None diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py index 8c6c1f9feb37..a104761b1acd 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py @@ -19,6 +19,7 @@ import unittest from dataclasses import dataclass from typing import Optional +from typing import Tuple from unittest.mock import MagicMock import pytest @@ -31,6 +32,7 @@ # pylint: disable=ungrouped-imports try: + from google.cloud.sql.connector import IPTypes from testcontainers.core.generic import DbContainer from testcontainers.postgres import PostgresContainer from testcontainers.mysql import MySqlContainer @@ -44,7 +46,11 @@ DatabaseTypeAdapter, CustomQueryConfig, TableFieldsQueryConfig, - TableFunctionQueryConfig) + TableFunctionQueryConfig, + CloudSQLConnectionConfig, + ExternalSQLDBConnectionConfig, + ConnectionConfig, + SQLClientConnectionHandler) except ImportError: raise unittest.SkipTest('Google Cloud SQL dependencies are not installed.') @@ -75,8 +81,7 @@ def address(self) -> str: @property def url(self) -> str: - dialect = self.adapter.to_sqlalchemy_dialect() - return f"{dialect}://{self.user}:{self.password}@{self.address}/{self.id}" + return self.adapter.to_sqlalchemy_dialect() + "://" class CloudSQLEnrichmentTestHelper: @@ -97,7 +102,7 @@ def start_sql_db_container( driver=database_type.value) sql_db_container.start() host = sql_db_container.get_container_host_ip() - port = sql_db_container.get_exposed_port(5432) + port = int(sql_db_container.get_exposed_port(5432)) elif database_type == DatabaseTypeAdapter.MYSQL: user, password, db_id = "test", "test", "test" @@ -109,7 +114,7 @@ def start_sql_db_container( MYSQL_DATABASE=db_id) sql_db_container.start() host = sql_db_container.get_container_host_ip() - port = sql_db_container.get_exposed_port(3306) + port = int(sql_db_container.get_exposed_port(3306)) elif database_type == DatabaseTypeAdapter.SQLSERVER: user, password, db_id = "SA", "A_Str0ng_Required_Password", "tempdb" @@ -121,7 +126,7 @@ def start_sql_db_container( dialect=database_type.to_sqlalchemy_dialect()) sql_db_container.start() host = sql_db_container.get_container_host_ip() - port = sql_db_container.get_exposed_port(1433) + port = int(sql_db_container.get_exposed_port(1433)) else: raise ValueError(f"Unsupported database type: {database_type}") @@ -170,11 +175,14 @@ def stop_sql_db_container(db_info: SQLDBContainerInfo): @staticmethod def create_table( table_id: str, - db_url: str, + connection_config: ConnectionConfig, columns: list[Column], table_data: list[dict], - metadata: MetaData = MetaData()) -> Engine: - engine = create_engine(db_url) + metadata: MetaData) -> Tuple[SQLClientConnectionHandler, Engine]: + sql_client_handler = connection_config.get_connector_handler() + engine = create_engine( + url=connection_config.get_db_url(), + creator=sql_client_handler.connector) table = Table(table_id, metadata, *columns) metadata.create_all(engine) @@ -188,15 +196,7 @@ def create_table( transaction.rollback() raise e - return engine - - -def init_db_type(db_type): - def wrapper(cls): - cls.db_type = db_type - return cls - - return wrapper + return sql_client_handler, engine @pytest.mark.uses_testcontainer @@ -227,17 +227,27 @@ class BaseTestCloudSQLEnrichment(unittest.TestCase): "id": 8, "name": "D", 'quantity': 4, 'distribution_center_id': 1 }, ] - db = None - _engine = None @classmethod def setUpClass(cls): - if not hasattr(cls, 'db_type'): + if not hasattr(cls, 'connection_config') or not hasattr(cls, '_metadata'): # Skip setup for the base class. - raise unittest.SkipTest("Base class - no db_type defined") - cls.db = CloudSQLEnrichmentTestHelper.start_sql_db_container(cls.db_type) - cls._engine = CloudSQLEnrichmentTestHelper.create_table( - cls._table_id, cls.db.url, cls.get_columns(), cls._table_data) + raise unittest.SkipTest( + "Base class - no connection_config or metadata defined") + + # Type hint data from subclasses. + cls._table_id: str + cls.connection_config: ConnectionConfig + cls._metadata: MetaData + + handler = CloudSQLEnrichmentTestHelper.create_table( + table_id=cls._table_id, + connection_config=cls.connection_config, + columns=cls.get_columns(), + table_data=cls._table_data, + metadata=cls._metadata) + + cls._sql_client_handler, cls._engine = handler cls._cache_client_retries = 3 @classmethod @@ -281,8 +291,8 @@ def _start_cache_container(self): @classmethod def tearDownClass(cls): + cls._metadata.drop_all(cls._engine) cls._engine.dispose(close=True) - CloudSQLEnrichmentTestHelper.stop_sql_db_container(cls.db) cls._engine = None def test_cloudsql_enrichment(self): @@ -302,11 +312,7 @@ def test_cloudsql_enrichment(self): where_clause_fields=fields) handler = CloudSQLEnrichmentHandler( - database_type_adapter=self.db.adapter, - database_address=self.db.address, - database_user=self.db.user, - database_password=self.db.id, - database_id=self.db.id, + connection_config=self.connection_config, query_config=query_config, min_batch_size=1, max_batch_size=100, @@ -333,11 +339,7 @@ def test_cloudsql_enrichment_batched(self): where_clause_fields=fields) handler = CloudSQLEnrichmentHandler( - database_type_adapter=self.db.adapter, - database_address=self.db.address, - database_user=self.db.user, - database_password=self.db.password, - database_id=self.db.id, + connection_config=self.connection_config, query_config=query_config, min_batch_size=2, max_batch_size=100, @@ -364,11 +366,7 @@ def test_cloudsql_enrichment_batched_multiple_fields(self): where_clause_fields=fields) handler = CloudSQLEnrichmentHandler( - database_type_adapter=self.db.adapter, - database_address=self.db.address, - database_user=self.db.user, - database_password=self.db.password, - database_id=self.db.id, + connection_config=self.connection_config, query_config=query_config, min_batch_size=8, max_batch_size=100, @@ -392,12 +390,7 @@ def test_cloudsql_enrichment_with_query_fn(self): query_config = CustomQueryConfig(query_fn=fn) handler = CloudSQLEnrichmentHandler( - database_type_adapter=self.db.adapter, - database_address=self.db.address, - database_user=self.db.user, - database_password=self.db.password, - database_id=self.db.id, - query_config=query_config) + connection_config=self.connection_config, query_config=query_config) with TestPipeline(is_integration_test=True) as test_pipeline: pcoll = (test_pipeline | beam.Create(requests) | Enrichment(handler)) @@ -419,11 +412,7 @@ def test_cloudsql_enrichment_with_condition_value_fn(self): where_clause_value_fn=where_clause_value_fn) handler = CloudSQLEnrichmentHandler( - database_type_adapter=self.db.adapter, - database_address=self.db.address, - database_user=self.db.user, - database_password=self.db.password, - database_id=self.db.id, + connection_config=self.connection_config, query_config=query_config, min_batch_size=2, max_batch_size=100) @@ -444,11 +433,7 @@ def test_cloudsql_enrichment_table_nonexistent_runtime_error_raised(self): where_clause_value_fn=where_clause_value_fn) handler = CloudSQLEnrichmentHandler( - database_type_adapter=self.db.adapter, - database_address=self.db.address, - database_user=self.db.user, - database_password=self.db.password, - database_id=self.db.id, + connection_config=self.connection_config, query_config=query_config, column_names=["wrong_column"], ) @@ -478,11 +463,7 @@ def test_cloudsql_enrichment_with_redis(self): where_clause_value_fn=where_clause_value_fn) handler = CloudSQLEnrichmentHandler( - database_type_adapter=self.db.adapter, - database_address=self.db.address, - database_user=self.db.user, - database_password=self.db.password, - database_id=self.db.id, + connection_config=self.connection_config, query_config=query_config, min_batch_size=2, max_batch_size=100) @@ -520,22 +501,104 @@ def test_cloudsql_enrichment_with_redis(self): CloudSQLEnrichmentHandler.__call__ = actual -@init_db_type(DatabaseTypeAdapter.POSTGRESQL) +class BaseCloudSQLDBEnrichment(BaseTestCloudSQLEnrichment): + @classmethod + def setUpClass(cls): + if not hasattr(cls, '_db_adapter'): + # Skip setup for the base class. + raise unittest.SkipTest("Base class - no db_adapter defined") + + # Type hint data from subclasses. + cls._db_adapter: DatabaseTypeAdapter + cls._instance_name: str + + # GCP project configuration. + gcp_project_id = "apache-beam-testing" + region = "us-central1" + + # Full instance connection name used for connecting via Cloud SQL. + instance_connection_name = f"{gcp_project_id}:{region}:{cls._instance_name}" + + # IAM user for Cloud SQL IAM DB authentication. + user = "beam" + + # Target database name within the Cloud SQL instance. + db_id = "testDB" + + # Type of IP address used to connect to the instance. + ip_type = IPTypes.PUBLIC + + cls.connection_config = CloudSQLConnectionConfig( + db_adapter=cls._db_adapter, + instance_connection_name=instance_connection_name, + user=user, + db_id=db_id, + enable_iam_auth=True, + password=None, + ip_type=ip_type) + super().setUpClass() + + @classmethod + def tearDownClass(cls): + super().tearDownClass() + cls._db = None + + +class TestCloudSQLPostgresEnrichment(BaseCloudSQLDBEnrichment): + _db_adapter = DatabaseTypeAdapter.POSTGRESQL + _instance_name = "test-postgres-instance" + _table_id = "product_details_cloudsql_pg_enrichment" + _metadata = MetaData() + + +@pytest.mark.uses_testcontainer +class BaseExternalSQLDBEnrichment(BaseTestCloudSQLEnrichment): + @classmethod + def setUpClass(cls): + if not hasattr(cls, '_db_adapter'): + # Skip setup for the base class. + raise unittest.SkipTest("Base class - no db_adapter defined") + + # Type hint data from subclasses. + cls._db_adapter: DatabaseTypeAdapter + + cls._db = CloudSQLEnrichmentTestHelper.start_sql_db_container( + cls._db_adapter) + cls.connection_config = ExternalSQLDBConnectionConfig( + db_adapter=cls._db_adapter, + host=cls._db.host, + user=cls._db.user, + password=cls._db.password, + db_id=cls._db.id, + port=cls._db.port) + super().setUpClass() + + @classmethod + def tearDownClass(cls): + super().tearDownClass() + CloudSQLEnrichmentTestHelper.stop_sql_db_container(cls._db) + cls._db = None + + @pytest.mark.uses_testcontainer -class TestCloudSQLEnrichmentPostgres(BaseTestCloudSQLEnrichment): - _table_id = "product_details_pg" +class TestExternalPostgresEnrichment(BaseExternalSQLDBEnrichment): + _db_adapter = DatabaseTypeAdapter.POSTGRESQL + _table_id = "product_details_external_pg_enrichment" + _metadata = MetaData() -@init_db_type(DatabaseTypeAdapter.MYSQL) @pytest.mark.uses_testcontainer -class TestCloudSQLEnrichmentMySQL(BaseTestCloudSQLEnrichment): - _table_id = "product_details_mysql" +class TestExternalMySQLEnrichment(BaseExternalSQLDBEnrichment): + _db_adapter = DatabaseTypeAdapter.MYSQL + _table_id = "product_details_external_mysql_enrichment" + _metadata = MetaData() -@init_db_type(DatabaseTypeAdapter.SQLSERVER) @pytest.mark.uses_testcontainer -class TestCloudSQLEnrichmentSQLServer(BaseTestCloudSQLEnrichment): - _table_id = "product_details_mssql" +class TestExternalSQLServerEnrichment(BaseExternalSQLDBEnrichment): + _db_adapter = DatabaseTypeAdapter.SQLSERVER + _table_id = "product_details_external_mssql_enrichment" + _metadata = MetaData() if __name__ == "__main__": diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_test.py index 30aeca4d04f6..6605430bcf27 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_test.py @@ -25,7 +25,8 @@ DatabaseTypeAdapter, CustomQueryConfig, TableFieldsQueryConfig, - TableFunctionQueryConfig) + TableFunctionQueryConfig, + ExternalSQLDBConnectionConfig) from apache_beam.transforms.enrichment_handlers.cloudsql_it_test import ( query_fn, where_clause_value_fn, @@ -95,19 +96,24 @@ class TestCloudSQLEnrichment(unittest.TestCase): def test_invalid_query_config( self, create_config, min_batch_size, max_batch_size, expected_error_msg): """Test that validation errors are raised for invalid query configs. - + The test verifies both that the appropriate ValueError is raised and that the error message contains the expected text. """ with self.assertRaises(ValueError) as context: # Call the lambda to create the config. query_config = create_config() + + connection_config = ExternalSQLDBConnectionConfig( + db_adapter=DatabaseTypeAdapter.POSTGRESQL, + host='', + port=5432, + user='', + password='', + db_id='') + _ = CloudSQLEnrichmentHandler( - database_type_adapter=DatabaseTypeAdapter.POSTGRESQL, - database_address='', - database_user='', - database_password='', - database_id='', + connection_config=connection_config, query_config=query_config, min_batch_size=min_batch_size, max_batch_size=max_batch_size, @@ -123,12 +129,16 @@ def test_valid_query_configs(self): where_clause_template="id = '{}'", where_clause_fields=["id"]) + connection_config = ExternalSQLDBConnectionConfig( + db_adapter=DatabaseTypeAdapter.POSTGRESQL, + host='localhost', + port=5432, + user='user', + password='password', + db_id='db') + handler1 = CloudSQLEnrichmentHandler( - database_type_adapter=DatabaseTypeAdapter.POSTGRESQL, - database_address='localhost', - database_user='user', - database_password='password', - database_id='db', + connection_config=connection_config, query_config=table_fields_config, min_batch_size=1, max_batch_size=10) @@ -143,11 +153,7 @@ def test_valid_query_configs(self): where_clause_value_fn=where_clause_value_fn) handler2 = CloudSQLEnrichmentHandler( - database_type_adapter=DatabaseTypeAdapter.POSTGRESQL, - database_address='localhost', - database_user='user', - database_password='password', - database_id='db', + connection_config=connection_config, query_config=table_function_config, min_batch_size=1, max_batch_size=10) @@ -159,12 +165,7 @@ def test_valid_query_configs(self): custom_config = CustomQueryConfig(query_fn=query_fn) handler3 = CloudSQLEnrichmentHandler( - database_type_adapter=DatabaseTypeAdapter.POSTGRESQL, - database_address='localhost', - database_user='user', - database_password='password', - database_id='db', - query_config=custom_config) + connection_config=connection_config, query_config=custom_config) # Verify that batching kwargs are empty for CustomQueryConfig. self.assertEqual(handler3.batch_elements_kwargs(), {}) @@ -173,13 +174,16 @@ def test_custom_query_config_cache_key_error(self): """Test get_cache_key raises NotImplementedError with CustomQueryConfig.""" custom_config = CustomQueryConfig(query_fn=query_fn) + connection_config = ExternalSQLDBConnectionConfig( + db_adapter=DatabaseTypeAdapter.POSTGRESQL, + host='localhost', + port=5432, + user='user', + password='password', + db_id='db') + handler = CloudSQLEnrichmentHandler( - database_type_adapter=DatabaseTypeAdapter.POSTGRESQL, - database_address='localhost', - database_user='user', - database_password='password', - database_id='db', - query_config=custom_config) + connection_config=connection_config, query_config=custom_config) # Create a dummy request import apache_beam as beam diff --git a/sdks/python/container/py310/base_image_requirements.txt b/sdks/python/container/py310/base_image_requirements.txt index 71dbfb4dd9cb..9914f2f30151 100644 --- a/sdks/python/container/py310/base_image_requirements.txt +++ b/sdks/python/container/py310/base_image_requirements.txt @@ -21,8 +21,13 @@ # https://s.apache.org/beam-python-dev-wiki # Reach out to a committer if you need help. +aiofiles==24.1.0 +aiohappyeyeballs==2.6.1 +aiohttp==3.12.12 +aiosignal==1.3.2 annotated-types==0.7.0 anyio==4.9.0 +asn1crypto==1.5.1 async-timeout==5.0.1 attrs==25.3.0 backports.tarfile==1.2.0 @@ -33,11 +38,11 @@ cachetools==5.5.2 certifi==2025.4.26 cffi==1.17.1 charset-normalizer==3.4.2 -click==8.2.0 +click==8.2.1 +cloud-sql-python-connector==1.18.2 crcmod==1.7 -cryptography==45.0.2 -Cython==3.1.1 -Deprecated==1.2.18 +cryptography==45.0.4 +Cython==3.1.2 deprecation==2.1.0 dill==0.3.1.1 dnspython==2.7.0 @@ -48,35 +53,36 @@ exceptiongroup==1.3.0 execnet==2.1.1 fastavro==1.11.1 fasteners==0.19 -freezegun==1.5.1 +freezegun==1.5.2 +frozenlist==1.7.0 future==1.0.0 -google-api-core==2.24.2 -google-api-python-client==2.169.0 +google-api-core==2.25.0 +google-api-python-client==2.172.0 google-apitools==0.5.31 -google-auth==2.40.1 +google-auth==2.40.3 google-auth-httplib2==0.2.0 -google-cloud-aiplatform==1.93.0 -google-cloud-bigquery==3.32.0 -google-cloud-bigquery-storage==2.31.0 -google-cloud-bigtable==2.30.1 +google-cloud-aiplatform==1.97.0 +google-cloud-bigquery==3.34.0 +google-cloud-bigquery-storage==2.32.0 +google-cloud-bigtable==2.31.0 google-cloud-core==2.4.3 google-cloud-datastore==2.21.0 -google-cloud-dlp==3.29.0 -google-cloud-language==2.17.1 +google-cloud-dlp==3.30.0 +google-cloud-language==2.17.2 google-cloud-profiler==4.1.0 -google-cloud-pubsub==2.29.0 +google-cloud-pubsub==2.30.0 google-cloud-pubsublite==1.12.0 -google-cloud-recommendations-ai==0.10.17 +google-cloud-recommendations-ai==0.10.18 google-cloud-resource-manager==1.14.2 -google-cloud-spanner==3.54.0 +google-cloud-spanner==3.55.0 google-cloud-storage==2.19.0 -google-cloud-videointelligence==2.16.1 -google-cloud-vision==3.10.1 +google-cloud-videointelligence==2.16.2 +google-cloud-vision==3.10.2 google-crc32c==1.7.1 -google-genai==1.15.0 +google-genai==1.20.0 google-resumable-media==2.7.2 googleapis-common-protos==1.70.0 -greenlet==3.2.2 +greenlet==3.2.3 grpc-google-iam-v1==0.14.2 grpc-interceptor==0.15.4 grpcio==1.65.5 @@ -87,70 +93,77 @@ hdfs==2.7.3 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -hypothesis==6.131.18 +hypothesis==6.135.7 idna==3.10 -importlib_metadata==8.6.1 +importlib_metadata==8.7.0 iniconfig==2.1.0 jaraco.classes==3.4.0 jaraco.context==6.0.1 jaraco.functools==4.1.0 jeepney==0.9.0 Jinja2==3.1.6 -joblib==1.5.0 +joblib==1.5.1 jsonpickle==3.4.2 -jsonschema==4.23.0 +jsonschema==4.24.0 jsonschema-specifications==2025.4.1 +kafka-python==2.2.11 keyring==25.6.0 keyrings.google-artifactregistry-auth==1.1.2 MarkupSafe==3.0.2 mmh3==5.1.0 mock==5.2.0 more-itertools==10.7.0 +multidict==6.4.4 +mysql-connector-python==9.3.0 nltk==3.9.1 numpy==2.2.6 oauth2client==4.1.3 objsize==0.7.1 -opentelemetry-api==1.33.1 -opentelemetry-sdk==1.33.1 -opentelemetry-semantic-conventions==0.54b1 +opentelemetry-api==1.34.1 +opentelemetry-sdk==1.34.1 +opentelemetry-semantic-conventions==0.55b1 orjson==3.10.18 overrides==7.7.0 packaging==25.0 pandas==2.2.3 parameterized==0.9.0 +pg8000==1.31.2 pluggy==1.6.0 +propcache==0.3.2 proto-plus==1.26.1 -protobuf==5.29.4 +protobuf==5.29.5 psycopg2-binary==2.9.10 pyarrow==18.1.0 pyarrow-hotfix==0.7 pyasn1==0.6.1 pyasn1_modules==0.4.2 pycparser==2.22 -pydantic==2.11.4 +pydantic==2.11.5 pydantic_core==2.33.2 pydot==1.4.2 PyHamcrest==2.1.0 PyJWT==2.9.0 -pymongo==4.13.0 +pymongo==4.13.1 PyMySQL==1.1.1 pyparsing==3.2.3 pyproject_hooks==1.2.0 pytest==7.4.4 pytest-timeout==2.4.0 -pytest-xdist==3.6.1 +pytest-xdist==3.7.0 python-dateutil==2.9.0.post0 +python-tds==1.16.1 pytz==2025.2 PyYAML==6.0.2 redis==5.3.0 referencing==0.36.2 regex==2024.11.6 -requests==2.32.3 +requests==2.32.4 requests-mock==1.12.1 -rpds-py==0.25.0 +rpds-py==0.25.1 rsa==4.9.1 -scikit-learn==1.6.1 +scikit-learn==1.7.0 scipy==1.15.3 +scramp==1.4.5 SecretStorage==3.3.3 shapely==2.1.1 six==1.17.0 @@ -158,19 +171,21 @@ sniffio==1.3.1 sortedcontainers==2.4.0 soupsieve==2.7 SQLAlchemy==2.0.41 +sqlalchemy_pytds==1.0.2 sqlparse==0.5.3 tenacity==8.5.0 testcontainers==3.7.1 threadpoolctl==3.6.0 tomli==2.2.1 tqdm==4.67.1 -typing-inspection==0.4.0 -typing_extensions==4.13.2 +typing-inspection==0.4.1 +typing_extensions==4.14.0 tzdata==2025.2 -uritemplate==4.1.1 +uritemplate==4.2.0 urllib3==2.4.0 virtualenv-clone==0.5.7 websockets==15.0.1 wrapt==1.17.2 -zipp==3.21.0 +yarl==1.20.1 +zipp==3.23.0 zstandard==0.23.0 diff --git a/sdks/python/container/py311/base_image_requirements.txt b/sdks/python/container/py311/base_image_requirements.txt index 4a818db73073..072846945690 100644 --- a/sdks/python/container/py311/base_image_requirements.txt +++ b/sdks/python/container/py311/base_image_requirements.txt @@ -21,8 +21,13 @@ # https://s.apache.org/beam-python-dev-wiki # Reach out to a committer if you need help. +aiofiles==24.1.0 +aiohappyeyeballs==2.6.1 +aiohttp==3.12.12 +aiosignal==1.3.2 annotated-types==0.7.0 anyio==4.9.0 +asn1crypto==1.5.1 attrs==25.3.0 backports.tarfile==1.2.0 beautifulsoup4==4.13.4 @@ -32,11 +37,11 @@ cachetools==5.5.2 certifi==2025.4.26 cffi==1.17.1 charset-normalizer==3.4.2 -click==8.2.0 +click==8.2.1 +cloud-sql-python-connector==1.18.2 crcmod==1.7 -cryptography==45.0.2 -Cython==3.1.1 -Deprecated==1.2.18 +cryptography==45.0.4 +Cython==3.1.2 deprecation==2.1.0 dill==0.3.1.1 dnspython==2.7.0 @@ -46,35 +51,36 @@ docstring_parser==0.16 execnet==2.1.1 fastavro==1.11.1 fasteners==0.19 -freezegun==1.5.1 +freezegun==1.5.2 +frozenlist==1.7.0 future==1.0.0 -google-api-core==2.24.2 -google-api-python-client==2.169.0 +google-api-core==2.25.0 +google-api-python-client==2.172.0 google-apitools==0.5.31 -google-auth==2.40.1 +google-auth==2.40.3 google-auth-httplib2==0.2.0 -google-cloud-aiplatform==1.93.0 -google-cloud-bigquery==3.32.0 -google-cloud-bigquery-storage==2.31.0 -google-cloud-bigtable==2.30.1 +google-cloud-aiplatform==1.97.0 +google-cloud-bigquery==3.34.0 +google-cloud-bigquery-storage==2.32.0 +google-cloud-bigtable==2.31.0 google-cloud-core==2.4.3 google-cloud-datastore==2.21.0 -google-cloud-dlp==3.29.0 -google-cloud-language==2.17.1 +google-cloud-dlp==3.30.0 +google-cloud-language==2.17.2 google-cloud-profiler==4.1.0 -google-cloud-pubsub==2.29.0 +google-cloud-pubsub==2.30.0 google-cloud-pubsublite==1.12.0 -google-cloud-recommendations-ai==0.10.17 +google-cloud-recommendations-ai==0.10.18 google-cloud-resource-manager==1.14.2 -google-cloud-spanner==3.54.0 +google-cloud-spanner==3.55.0 google-cloud-storage==2.19.0 -google-cloud-videointelligence==2.16.1 -google-cloud-vision==3.10.1 +google-cloud-videointelligence==2.16.2 +google-cloud-vision==3.10.2 google-crc32c==1.7.1 -google-genai==1.15.0 +google-genai==1.20.0 google-resumable-media==2.7.2 googleapis-common-protos==1.70.0 -greenlet==3.2.2 +greenlet==3.2.3 grpc-google-iam-v1==0.14.2 grpc-interceptor==0.15.4 grpcio==1.65.5 @@ -85,70 +91,77 @@ hdfs==2.7.3 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -hypothesis==6.131.18 +hypothesis==6.135.7 idna==3.10 -importlib_metadata==8.6.1 +importlib_metadata==8.7.0 iniconfig==2.1.0 jaraco.classes==3.4.0 jaraco.context==6.0.1 jaraco.functools==4.1.0 jeepney==0.9.0 Jinja2==3.1.6 -joblib==1.5.0 +joblib==1.5.1 jsonpickle==3.4.2 -jsonschema==4.23.0 +jsonschema==4.24.0 jsonschema-specifications==2025.4.1 +kafka-python==2.2.11 keyring==25.6.0 keyrings.google-artifactregistry-auth==1.1.2 MarkupSafe==3.0.2 mmh3==5.1.0 mock==5.2.0 more-itertools==10.7.0 +multidict==6.4.4 +mysql-connector-python==9.3.0 nltk==3.9.1 numpy==2.2.6 oauth2client==4.1.3 objsize==0.7.1 -opentelemetry-api==1.33.1 -opentelemetry-sdk==1.33.1 -opentelemetry-semantic-conventions==0.54b1 +opentelemetry-api==1.34.1 +opentelemetry-sdk==1.34.1 +opentelemetry-semantic-conventions==0.55b1 orjson==3.10.18 overrides==7.7.0 packaging==25.0 pandas==2.2.3 parameterized==0.9.0 +pg8000==1.31.2 pluggy==1.6.0 +propcache==0.3.2 proto-plus==1.26.1 -protobuf==5.29.4 +protobuf==5.29.5 psycopg2-binary==2.9.10 pyarrow==18.1.0 pyarrow-hotfix==0.7 pyasn1==0.6.1 pyasn1_modules==0.4.2 pycparser==2.22 -pydantic==2.11.4 +pydantic==2.11.5 pydantic_core==2.33.2 pydot==1.4.2 PyHamcrest==2.1.0 PyJWT==2.9.0 -pymongo==4.13.0 +pymongo==4.13.1 PyMySQL==1.1.1 pyparsing==3.2.3 pyproject_hooks==1.2.0 pytest==7.4.4 pytest-timeout==2.4.0 -pytest-xdist==3.6.1 +pytest-xdist==3.7.0 python-dateutil==2.9.0.post0 +python-tds==1.16.1 pytz==2025.2 PyYAML==6.0.2 redis==5.3.0 referencing==0.36.2 regex==2024.11.6 -requests==2.32.3 +requests==2.32.4 requests-mock==1.12.1 -rpds-py==0.25.0 +rpds-py==0.25.1 rsa==4.9.1 -scikit-learn==1.6.1 +scikit-learn==1.7.0 scipy==1.15.3 +scramp==1.4.5 SecretStorage==3.3.3 shapely==2.1.1 six==1.17.0 @@ -156,18 +169,20 @@ sniffio==1.3.1 sortedcontainers==2.4.0 soupsieve==2.7 SQLAlchemy==2.0.41 +sqlalchemy_pytds==1.0.2 sqlparse==0.5.3 tenacity==8.5.0 testcontainers==3.7.1 threadpoolctl==3.6.0 tqdm==4.67.1 -typing-inspection==0.4.0 -typing_extensions==4.13.2 +typing-inspection==0.4.1 +typing_extensions==4.14.0 tzdata==2025.2 -uritemplate==4.1.1 +uritemplate==4.2.0 urllib3==2.4.0 virtualenv-clone==0.5.7 websockets==15.0.1 wrapt==1.17.2 -zipp==3.21.0 +yarl==1.20.1 +zipp==3.23.0 zstandard==0.23.0 diff --git a/sdks/python/container/py312/base_image_requirements.txt b/sdks/python/container/py312/base_image_requirements.txt index 44a3e8d21046..6d5c14552656 100644 --- a/sdks/python/container/py312/base_image_requirements.txt +++ b/sdks/python/container/py312/base_image_requirements.txt @@ -21,8 +21,13 @@ # https://s.apache.org/beam-python-dev-wiki # Reach out to a committer if you need help. +aiofiles==24.1.0 +aiohappyeyeballs==2.6.1 +aiohttp==3.12.12 +aiosignal==1.3.2 annotated-types==0.7.0 anyio==4.9.0 +asn1crypto==1.5.1 attrs==25.3.0 beautifulsoup4==4.13.4 bs4==0.0.2 @@ -31,11 +36,11 @@ cachetools==5.5.2 certifi==2025.4.26 cffi==1.17.1 charset-normalizer==3.4.2 -click==8.2.0 +click==8.2.1 +cloud-sql-python-connector==1.18.2 crcmod==1.7 -cryptography==45.0.2 -Cython==3.1.1 -Deprecated==1.2.18 +cryptography==45.0.4 +Cython==3.1.2 deprecation==2.1.0 dill==0.3.1.1 dnspython==2.7.0 @@ -45,35 +50,36 @@ docstring_parser==0.16 execnet==2.1.1 fastavro==1.11.1 fasteners==0.19 -freezegun==1.5.1 +freezegun==1.5.2 +frozenlist==1.7.0 future==1.0.0 -google-api-core==2.24.2 -google-api-python-client==2.169.0 +google-api-core==2.25.0 +google-api-python-client==2.172.0 google-apitools==0.5.31 -google-auth==2.40.1 +google-auth==2.40.3 google-auth-httplib2==0.2.0 -google-cloud-aiplatform==1.93.0 -google-cloud-bigquery==3.32.0 -google-cloud-bigquery-storage==2.31.0 -google-cloud-bigtable==2.30.1 +google-cloud-aiplatform==1.97.0 +google-cloud-bigquery==3.34.0 +google-cloud-bigquery-storage==2.32.0 +google-cloud-bigtable==2.31.0 google-cloud-core==2.4.3 google-cloud-datastore==2.21.0 -google-cloud-dlp==3.29.0 -google-cloud-language==2.17.1 +google-cloud-dlp==3.30.0 +google-cloud-language==2.17.2 google-cloud-profiler==4.1.0 -google-cloud-pubsub==2.29.0 +google-cloud-pubsub==2.30.0 google-cloud-pubsublite==1.12.0 -google-cloud-recommendations-ai==0.10.17 +google-cloud-recommendations-ai==0.10.18 google-cloud-resource-manager==1.14.2 -google-cloud-spanner==3.54.0 +google-cloud-spanner==3.55.0 google-cloud-storage==2.19.0 -google-cloud-videointelligence==2.16.1 -google-cloud-vision==3.10.1 +google-cloud-videointelligence==2.16.2 +google-cloud-vision==3.10.2 google-crc32c==1.7.1 -google-genai==1.15.0 +google-genai==1.20.0 google-resumable-media==2.7.2 googleapis-common-protos==1.70.0 -greenlet==3.2.2 +greenlet==3.2.3 grpc-google-iam-v1==0.14.2 grpc-interceptor==0.15.4 grpcio==1.65.5 @@ -84,91 +90,100 @@ hdfs==2.7.3 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -hypothesis==6.131.18 +hypothesis==6.135.7 idna==3.10 -importlib_metadata==8.6.1 +importlib_metadata==8.7.0 iniconfig==2.1.0 jaraco.classes==3.4.0 jaraco.context==6.0.1 jaraco.functools==4.1.0 jeepney==0.9.0 Jinja2==3.1.6 -joblib==1.5.0 +joblib==1.5.1 jsonpickle==3.4.2 -jsonschema==4.23.0 +jsonschema==4.24.0 jsonschema-specifications==2025.4.1 +kafka-python==2.2.11 keyring==25.6.0 keyrings.google-artifactregistry-auth==1.1.2 MarkupSafe==3.0.2 mmh3==5.1.0 mock==5.2.0 more-itertools==10.7.0 +multidict==6.4.4 +mysql-connector-python==9.3.0 nltk==3.9.1 numpy==2.2.6 oauth2client==4.1.3 objsize==0.7.1 -opentelemetry-api==1.33.1 -opentelemetry-sdk==1.33.1 -opentelemetry-semantic-conventions==0.54b1 +opentelemetry-api==1.34.1 +opentelemetry-sdk==1.34.1 +opentelemetry-semantic-conventions==0.55b1 orjson==3.10.18 overrides==7.7.0 packaging==25.0 pandas==2.2.3 parameterized==0.9.0 +pg8000==1.31.2 pluggy==1.6.0 +propcache==0.3.2 proto-plus==1.26.1 -protobuf==5.29.4 +protobuf==5.29.5 psycopg2-binary==2.9.10 pyarrow==18.1.0 pyarrow-hotfix==0.7 pyasn1==0.6.1 pyasn1_modules==0.4.2 pycparser==2.22 -pydantic==2.11.4 +pydantic==2.11.5 pydantic_core==2.33.2 pydot==1.4.2 PyHamcrest==2.1.0 PyJWT==2.9.0 -pymongo==4.13.0 +pymongo==4.13.1 PyMySQL==1.1.1 pyparsing==3.2.3 pyproject_hooks==1.2.0 pytest==7.4.4 pytest-timeout==2.4.0 -pytest-xdist==3.6.1 +pytest-xdist==3.7.0 python-dateutil==2.9.0.post0 +python-tds==1.16.1 pytz==2025.2 PyYAML==6.0.2 redis==5.3.0 referencing==0.36.2 regex==2024.11.6 -requests==2.32.3 +requests==2.32.4 requests-mock==1.12.1 -rpds-py==0.25.0 +rpds-py==0.25.1 rsa==4.9.1 -scikit-learn==1.6.1 +scikit-learn==1.7.0 scipy==1.15.3 +scramp==1.4.5 SecretStorage==3.3.3 -setuptools==80.7.1 +setuptools==80.9.0 shapely==2.1.1 six==1.17.0 sniffio==1.3.1 sortedcontainers==2.4.0 soupsieve==2.7 SQLAlchemy==2.0.41 +sqlalchemy_pytds==1.0.2 sqlparse==0.5.3 tenacity==8.5.0 testcontainers==3.7.1 threadpoolctl==3.6.0 tqdm==4.67.1 -typing-inspection==0.4.0 -typing_extensions==4.13.2 +typing-inspection==0.4.1 +typing_extensions==4.14.0 tzdata==2025.2 -uritemplate==4.1.1 +uritemplate==4.2.0 urllib3==2.4.0 virtualenv-clone==0.5.7 websockets==15.0.1 wheel==0.45.1 wrapt==1.17.2 -zipp==3.21.0 +yarl==1.20.1 +zipp==3.23.0 zstandard==0.23.0 diff --git a/sdks/python/container/py313/base_image_requirements.txt b/sdks/python/container/py313/base_image_requirements.txt index 3c93397afb1a..ea856d786160 100644 --- a/sdks/python/container/py313/base_image_requirements.txt +++ b/sdks/python/container/py313/base_image_requirements.txt @@ -21,8 +21,13 @@ # https://s.apache.org/beam-python-dev-wiki # Reach out to a committer if you need help. +aiofiles==24.1.0 +aiohappyeyeballs==2.6.1 +aiohttp==3.12.12 +aiosignal==1.3.2 annotated-types==0.7.0 anyio==4.9.0 +asn1crypto==1.5.1 attrs==25.3.0 beautifulsoup4==4.13.4 bs4==0.0.2 @@ -31,11 +36,11 @@ cachetools==5.5.2 certifi==2025.4.26 cffi==1.17.1 charset-normalizer==3.4.2 -click==8.2.0 +click==8.2.1 +cloud-sql-python-connector==1.18.2 crcmod==1.7 -cryptography==45.0.2 -Cython==3.1.1 -Deprecated==1.2.18 +cryptography==45.0.4 +Cython==3.1.2 deprecation==2.1.0 dill==0.3.1.1 dnspython==2.7.0 @@ -45,36 +50,37 @@ docstring_parser==0.16 execnet==2.1.1 fastavro==1.11.1 fasteners==0.19 -freezegun==1.5.1 +freezegun==1.5.2 +frozenlist==1.7.0 future==1.0.0 -google-api-core==2.24.2 +google-api-core==2.25.0 google-apitools==0.5.32 -google-auth==2.40.1 +google-auth==2.40.3 google-auth-httplib2==0.2.0 -google-cloud-aiplatform==1.93.1 -google-cloud-bigquery==3.33.0 -google-cloud-bigquery-storage==2.31.0 -google-cloud-bigtable==2.30.1 +google-cloud-aiplatform==1.97.0 +google-cloud-bigquery==3.34.0 +google-cloud-bigquery-storage==2.32.0 +google-cloud-bigtable==2.31.0 google-cloud-core==2.4.3 google-cloud-datastore==2.21.0 -google-cloud-dlp==3.29.0 -google-cloud-language==2.17.1 -google-cloud-pubsub==2.29.0 +google-cloud-dlp==3.30.0 +google-cloud-language==2.17.2 +google-cloud-pubsub==2.30.0 google-cloud-pubsublite==1.12.0 -google-cloud-recommendations-ai==0.10.17 +google-cloud-recommendations-ai==0.10.18 google-cloud-resource-manager==1.14.2 -google-cloud-spanner==3.54.0 +google-cloud-spanner==3.55.0 google-cloud-storage==2.19.0 -google-cloud-videointelligence==2.16.1 -google-cloud-vision==3.10.1 +google-cloud-videointelligence==2.16.2 +google-cloud-vision==3.10.2 google-crc32c==1.7.1 -google-genai==1.16.1 +google-genai==1.20.0 google-resumable-media==2.7.2 googleapis-common-protos==1.70.0 -greenlet==3.2.2 +greenlet==3.2.3 grpc-google-iam-v1==0.14.2 grpc-interceptor==0.15.4 -grpcio==1.71.0 +grpcio==1.73.0 grpcio-status==1.71.0 guppy3==3.1.5 h11==0.16.0 @@ -82,90 +88,99 @@ hdfs==2.7.3 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -hypothesis==6.131.20 +hypothesis==6.135.7 idna==3.10 -importlib_metadata==8.6.1 +importlib_metadata==8.7.0 iniconfig==2.1.0 jaraco.classes==3.4.0 jaraco.context==6.0.1 jaraco.functools==4.1.0 jeepney==0.9.0 Jinja2==3.1.6 -joblib==1.5.0 +joblib==1.5.1 jsonpickle==3.4.2 -jsonschema==4.23.0 +jsonschema==4.24.0 jsonschema-specifications==2025.4.1 +kafka-python==2.2.11 keyring==25.6.0 keyrings.google-artifactregistry-auth==1.1.2 MarkupSafe==3.0.2 mmh3==5.1.0 mock==5.2.0 more-itertools==10.7.0 +multidict==6.4.4 +mysql-connector-python==9.3.0 nltk==3.9.1 numpy==2.2.6 oauth2client==4.1.3 objsize==0.7.1 -opentelemetry-api==1.33.1 -opentelemetry-sdk==1.33.1 -opentelemetry-semantic-conventions==0.54b1 +opentelemetry-api==1.34.1 +opentelemetry-sdk==1.34.1 +opentelemetry-semantic-conventions==0.55b1 orjson==3.10.18 overrides==7.7.0 packaging==25.0 pandas==2.2.3 parameterized==0.9.0 +pg8000==1.31.2 pluggy==1.6.0 +propcache==0.3.2 proto-plus==1.26.1 -protobuf==5.29.4 +protobuf==5.29.5 psycopg2-binary==2.9.10 pyarrow==18.1.0 pyarrow-hotfix==0.7 pyasn1==0.6.1 pyasn1_modules==0.4.2 pycparser==2.22 -pydantic==2.11.4 +pydantic==2.11.5 pydantic_core==2.33.2 pydot==1.4.2 PyHamcrest==2.1.0 PyJWT==2.9.0 -pymongo==4.13.0 +pymongo==4.13.1 PyMySQL==1.1.1 pyparsing==3.2.3 pyproject_hooks==1.2.0 pytest==7.4.4 pytest-timeout==2.4.0 -pytest-xdist==3.6.1 +pytest-xdist==3.7.0 python-dateutil==2.9.0.post0 +python-tds==1.16.1 pytz==2025.2 PyYAML==6.0.2 redis==5.3.0 referencing==0.36.2 regex==2024.11.6 -requests==2.32.3 +requests==2.32.4 requests-mock==1.12.1 -rpds-py==0.25.0 +rpds-py==0.25.1 rsa==4.9.1 -scikit-learn==1.6.1 +scikit-learn==1.7.0 scipy==1.15.3 +scramp==1.4.5 SecretStorage==3.3.3 -setuptools==80.8.0 +setuptools==80.9.0 shapely==2.1.1 six==1.17.0 sniffio==1.3.1 sortedcontainers==2.4.0 soupsieve==2.7 SQLAlchemy==2.0.41 +sqlalchemy_pytds==1.0.2 sqlparse==0.5.3 tenacity==8.5.0 testcontainers==3.7.1 threadpoolctl==3.6.0 tqdm==4.67.1 -typing-inspection==0.4.0 -typing_extensions==4.13.2 +typing-inspection==0.4.1 +typing_extensions==4.14.0 tzdata==2025.2 urllib3==2.4.0 virtualenv-clone==0.5.7 websockets==15.0.1 wheel==0.45.1 wrapt==1.17.2 -zipp==3.21.0 +yarl==1.20.1 +zipp==3.23.0 zstandard==0.23.0 diff --git a/sdks/python/container/py39/base_image_requirements.txt b/sdks/python/container/py39/base_image_requirements.txt index 6591f108a99e..31104e207fda 100644 --- a/sdks/python/container/py39/base_image_requirements.txt +++ b/sdks/python/container/py39/base_image_requirements.txt @@ -21,8 +21,13 @@ # https://s.apache.org/beam-python-dev-wiki # Reach out to a committer if you need help. +aiofiles==24.1.0 +aiohappyeyeballs==2.6.1 +aiohttp==3.12.12 +aiosignal==1.3.2 annotated-types==0.7.0 anyio==4.9.0 +asn1crypto==1.5.1 async-timeout==5.0.1 attrs==25.3.0 backports.tarfile==1.2.0 @@ -34,10 +39,10 @@ certifi==2025.4.26 cffi==1.17.1 charset-normalizer==3.4.2 click==8.1.8 +cloud-sql-python-connector==1.18.2 crcmod==1.7 -cryptography==45.0.2 -Cython==3.1.1 -Deprecated==1.2.18 +cryptography==45.0.4 +Cython==3.1.2 deprecation==2.1.0 dill==0.3.1.1 dnspython==2.7.0 @@ -48,35 +53,36 @@ exceptiongroup==1.3.0 execnet==2.1.1 fastavro==1.11.1 fasteners==0.19 -freezegun==1.5.1 +freezegun==1.5.2 +frozenlist==1.7.0 future==1.0.0 -google-api-core==2.24.2 -google-api-python-client==2.169.0 +google-api-core==2.25.0 +google-api-python-client==2.172.0 google-apitools==0.5.31 -google-auth==2.40.1 +google-auth==2.40.3 google-auth-httplib2==0.2.0 -google-cloud-aiplatform==1.93.0 -google-cloud-bigquery==3.32.0 -google-cloud-bigquery-storage==2.31.0 -google-cloud-bigtable==2.30.1 +google-cloud-aiplatform==1.97.0 +google-cloud-bigquery==3.34.0 +google-cloud-bigquery-storage==2.32.0 +google-cloud-bigtable==2.31.0 google-cloud-core==2.4.3 google-cloud-datastore==2.21.0 -google-cloud-dlp==3.29.0 -google-cloud-language==2.17.1 +google-cloud-dlp==3.30.0 +google-cloud-language==2.17.2 google-cloud-profiler==4.1.0 -google-cloud-pubsub==2.29.0 +google-cloud-pubsub==2.30.0 google-cloud-pubsublite==1.12.0 -google-cloud-recommendations-ai==0.10.17 +google-cloud-recommendations-ai==0.10.18 google-cloud-resource-manager==1.14.2 -google-cloud-spanner==3.54.0 +google-cloud-spanner==3.55.0 google-cloud-storage==2.19.0 -google-cloud-videointelligence==2.16.1 -google-cloud-vision==3.10.1 +google-cloud-videointelligence==2.16.2 +google-cloud-vision==3.10.2 google-crc32c==1.7.1 -google-genai==1.15.0 +google-genai==1.20.0 google-resumable-media==2.7.2 googleapis-common-protos==1.70.0 -greenlet==3.2.2 +greenlet==3.2.3 grpc-google-iam-v1==0.14.2 grpc-interceptor==0.15.4 grpcio==1.65.5 @@ -87,70 +93,77 @@ hdfs==2.7.3 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -hypothesis==6.131.18 +hypothesis==6.135.7 idna==3.10 -importlib_metadata==8.6.1 +importlib_metadata==8.7.0 iniconfig==2.1.0 jaraco.classes==3.4.0 jaraco.context==6.0.1 jaraco.functools==4.1.0 jeepney==0.9.0 Jinja2==3.1.6 -joblib==1.5.0 +joblib==1.5.1 jsonpickle==3.4.2 -jsonschema==4.23.0 +jsonschema==4.24.0 jsonschema-specifications==2025.4.1 +kafka-python==2.2.11 keyring==25.6.0 keyrings.google-artifactregistry-auth==1.1.2 MarkupSafe==3.0.2 mmh3==5.1.0 mock==5.2.0 more-itertools==10.7.0 +multidict==6.4.4 +mysql-connector-python==9.3.0 nltk==3.9.1 numpy==2.0.2 oauth2client==4.1.3 objsize==0.7.1 -opentelemetry-api==1.33.1 -opentelemetry-sdk==1.33.1 -opentelemetry-semantic-conventions==0.54b1 +opentelemetry-api==1.34.1 +opentelemetry-sdk==1.34.1 +opentelemetry-semantic-conventions==0.55b1 orjson==3.10.18 overrides==7.7.0 packaging==25.0 pandas==2.2.3 parameterized==0.9.0 +pg8000==1.31.2 pluggy==1.6.0 +propcache==0.3.2 proto-plus==1.26.1 -protobuf==5.29.4 +protobuf==5.29.5 psycopg2-binary==2.9.9 pyarrow==18.1.0 pyarrow-hotfix==0.7 pyasn1==0.6.1 pyasn1_modules==0.4.2 pycparser==2.22 -pydantic==2.11.4 +pydantic==2.11.5 pydantic_core==2.33.2 pydot==1.4.2 PyHamcrest==2.1.0 PyJWT==2.9.0 -pymongo==4.13.0 +pymongo==4.13.1 PyMySQL==1.1.1 pyparsing==3.2.3 pyproject_hooks==1.2.0 pytest==7.4.4 pytest-timeout==2.4.0 -pytest-xdist==3.6.1 +pytest-xdist==3.7.0 python-dateutil==2.9.0.post0 +python-tds==1.16.1 pytz==2025.2 PyYAML==6.0.2 redis==5.3.0 referencing==0.36.2 regex==2024.11.6 -requests==2.32.3 +requests==2.32.4 requests-mock==1.12.1 -rpds-py==0.25.0 +rpds-py==0.25.1 rsa==4.9.1 scikit-learn==1.6.1 scipy==1.13.1 +scramp==1.4.5 SecretStorage==3.3.3 shapely==2.0.7 six==1.17.0 @@ -158,19 +171,21 @@ sniffio==1.3.1 sortedcontainers==2.4.0 soupsieve==2.7 SQLAlchemy==2.0.41 +sqlalchemy_pytds==1.0.2 sqlparse==0.5.3 tenacity==8.5.0 testcontainers==3.7.1 threadpoolctl==3.6.0 tomli==2.2.1 tqdm==4.67.1 -typing-inspection==0.4.0 -typing_extensions==4.13.2 +typing-inspection==0.4.1 +typing_extensions==4.14.0 tzdata==2025.2 -uritemplate==4.1.1 +uritemplate==4.2.0 urllib3==2.4.0 virtualenv-clone==0.5.7 websockets==15.0.1 wrapt==1.17.2 -zipp==3.21.0 +yarl==1.20.1 +zipp==3.23.0 zstandard==0.23.0 diff --git a/sdks/python/setup.py b/sdks/python/setup.py index a98eaab33361..31186a7b1e8e 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -436,7 +436,9 @@ def get_portability_package_data(): 'virtualenv-clone>=0.5,<1.0', 'mysql-connector-python>=9.3.0', 'python-tds>=1.16.1', - 'sqlalchemy-pytds>=1.0.2' + 'sqlalchemy-pytds>=1.0.2', + 'pg8000>=1.31.1', + "PyMySQL>=1.1.0" ], 'gcp': [ 'cachetools>=3.1.0,<6', @@ -465,6 +467,10 @@ def get_portability_package_data(): 'google-cloud-vision>=2,<4', 'google-cloud-recommendations-ai>=0.1.0,<0.11.0', 'google-cloud-aiplatform>=1.26.0, < 2.0', + 'cloud-sql-python-connector>=1.18.2,<2.0.0', + 'python-tds>=1.16.1', + 'pg8000>=1.31.1', + "PyMySQL>=1.1.0", # Authentication for Google Artifact Registry when using # --extra-index-url or --index-url in requirements.txt in # Dataflow, which allows installing python packages from private From 5a472b7a11076bb4b2eed1e711d94a42c44258ca Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Fri, 13 Jun 2025 08:13:01 +0000 Subject: [PATCH 20/83] fix setup.py --- sdks/python/setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/setup.py b/sdks/python/setup.py index 39c5a23d0629..af9e5e5019fd 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -446,7 +446,7 @@ def get_portability_package_data(): 'python-tds>=1.16.1', 'sqlalchemy-pytds>=1.0.2', 'pg8000>=1.31.1', - "PyMySQL>=1.1.0" + "PyMySQL>=1.1.0", 'oracledb>=3.1.1' ], 'gcp': [ From 1083b184af7326128b0edbc7060364296b684b1f Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Fri, 27 Jun 2025 23:24:28 +0000 Subject: [PATCH 21/83] sdks/python: fix linting issues --- .../transforms/enrichment_handlers/cloudsql.py | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py index 6acb437cb782..1f62b052cddd 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py @@ -138,7 +138,7 @@ class CloudSQLConnectionConfig(ConnectionConfig): def __init__( self, db_adapter: DatabaseTypeAdapter, - instance_connection_name: str, + instance_connection_uri: str, user: str, db_id: str, enable_iam_auth: bool = True, @@ -146,20 +146,20 @@ def __init__( ip_type: IPTypes = IPTypes.PUBLIC, refresh_strategy: RefreshStrategy = RefreshStrategy.LAZY, *, - connector_kwargs: dict = {}, - connect_kwargs: dict = {}, + connector_kwargs: Optional[dict] = None, + connect_kwargs: Optional[dict] = None, ): self._validate_metadata(db_adapter=db_adapter) self._db_adapter = db_adapter - self._instance_connection_name = instance_connection_name + self._instance_connection_uri = instance_connection_uri self._user = user self._db_id = db_id self._enable_iam_auth = enable_iam_auth self._password = password self._ip_type = ip_type self._refresh_strategy = refresh_strategy - self.connector_kwargs = connector_kwargs - self.connect_kwargs = connect_kwargs + self.connector_kwargs = connector_kwargs or {} + self.connect_kwargs = connect_kwargs or {} def get_connector_handler(self) -> SQLClientConnectionHandler: cloudsql_client = CloudSQLConnector( @@ -168,7 +168,7 @@ def get_connector_handler(self) -> SQLClientConnectionHandler: **self.connector_kwargs) cloudsql_connector = lambda: cloudsql_client.connect( - instance_connection_string=self._instance_connection_name, driver=self. + instance_connection_string=self._instance_connection_uri, driver=self. _db_adapter.value, user=self._user, db=self._db_id, enable_iam_auth=self ._enable_iam_auth, password=self.password, **self.connect_kwargs) From fe6d7d03459b63d3cc3dc94eea447946ea9f2aa8 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Fri, 27 Jun 2025 23:24:44 +0000 Subject: [PATCH 22/83] sdks/python: connect to beam cloudsql itest instance --- .../enrichment_handlers/cloudsql_it_test.py | 30 +++++++++---------- 1 file changed, 15 insertions(+), 15 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py index a104761b1acd..bf08a82bcba5 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py @@ -16,6 +16,7 @@ # import functools import logging +import os import unittest from dataclasses import dataclass from typing import Optional @@ -62,7 +63,7 @@ def where_clause_value_fn(row: beam.Row): def query_fn(table, row: beam.Row): - return f"SELECT * FROM `{table}` WHERE id = {row.id}" # type: ignore[attr-defined] + return f"SELECT * FROM {table} WHERE id = {row.id}" # type: ignore[attr-defined] @dataclass @@ -471,7 +472,8 @@ def test_cloudsql_enrichment_with_redis(self): pcoll_populate_cache = ( test_pipeline | beam.Create(requests) - | Enrichment(handler).with_redis_cache(self.host, self.port)) + | Enrichment(handler).with_redis_cache( + self._cache_container_host, self._cache_container_port)) assert_that(pcoll_populate_cache, equal_to(expected_rows)) @@ -493,7 +495,8 @@ def test_cloudsql_enrichment_with_redis(self): pcoll_cached = ( test_pipeline | beam.Create(requests) - | Enrichment(handler).with_redis_cache(self.host, self.port)) + | Enrichment(handler).with_redis_cache( + self._cache_container_host, self._cache_container_port)) assert_that(pcoll_cached, equal_to(expected_rows)) @@ -517,25 +520,22 @@ def setUpClass(cls): region = "us-central1" # Full instance connection name used for connecting via Cloud SQL. - instance_connection_name = f"{gcp_project_id}:{region}:{cls._instance_name}" + instance_connection_name = f"{gcp_project_id}:{region}:{cls._instance_uri}" - # IAM user for Cloud SQL IAM DB authentication. - user = "beam" + # Password auth configuration for CloudSQL instance. + user = "postgres" + password = os.getenv("ALLOYDB_PASSWORD") - # Target database name within the Cloud SQL instance. - db_id = "testDB" - - # Type of IP address used to connect to the instance. - ip_type = IPTypes.PUBLIC + # Database ID for CloudSQL instance. + db_id = "postgres" cls.connection_config = CloudSQLConnectionConfig( db_adapter=cls._db_adapter, instance_connection_name=instance_connection_name, user=user, + password=password, db_id=db_id, - enable_iam_auth=True, - password=None, - ip_type=ip_type) + enable_iam_auth=False) super().setUpClass() @classmethod @@ -546,7 +546,7 @@ def tearDownClass(cls): class TestCloudSQLPostgresEnrichment(BaseCloudSQLDBEnrichment): _db_adapter = DatabaseTypeAdapter.POSTGRESQL - _instance_name = "test-postgres-instance" + _instance_name = "beam-integration-tests" _table_id = "product_details_cloudsql_pg_enrichment" _metadata = MetaData() From 0e898199dbb86621563428494780869bff74b685 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Fri, 27 Jun 2025 23:25:10 +0000 Subject: [PATCH 23/83] .github: trigger postcommits python --- .github/trigger_files/beam_PostCommit_Python.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/trigger_files/beam_PostCommit_Python.json b/.github/trigger_files/beam_PostCommit_Python.json index ca5f501f38d9..3f407dbe9466 100644 --- a/.github/trigger_files/beam_PostCommit_Python.json +++ b/.github/trigger_files/beam_PostCommit_Python.json @@ -1,5 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run.", - "modification": 14 + "modification": 15 } From ef106550721d0f4914309dc17969a9b3fa5fb9ba Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Fri, 27 Jun 2025 23:43:55 +0000 Subject: [PATCH 24/83] sdks/python: update base image requirements --- .../py310/base_image_requirements.txt | 34 +++++---------- .../py311/base_image_requirements.txt | 39 ++++++------------ .../py312/base_image_requirements.txt | 39 ++++++------------ .../py39/base_image_requirements.txt | 41 +++++-------------- 4 files changed, 46 insertions(+), 107 deletions(-) diff --git a/sdks/python/container/py310/base_image_requirements.txt b/sdks/python/container/py310/base_image_requirements.txt index 8fbddb2ce417..219a80863c78 100644 --- a/sdks/python/container/py310/base_image_requirements.txt +++ b/sdks/python/container/py310/base_image_requirements.txt @@ -23,11 +23,7 @@ aiofiles==24.1.0 aiohappyeyeballs==2.6.1 -<<<<<<< HEAD -aiohttp==3.12.12 -======= aiohttp==3.12.13 ->>>>>>> upstream/master aiosignal==1.3.2 annotated-types==0.7.0 anyio==4.9.0 @@ -60,22 +56,18 @@ fasteners==0.19 freezegun==1.5.2 frozenlist==1.7.0 future==1.0.0 -<<<<<<< HEAD -google-api-core==2.25.0 -======= google-api-core==2.25.1 ->>>>>>> upstream/master -google-api-python-client==2.172.0 +google-api-python-client==2.174.0 google-apitools==0.5.31 google-auth==2.40.3 google-auth-httplib2==0.2.0 -google-cloud-aiplatform==1.97.0 +google-cloud-aiplatform==1.100.0 google-cloud-bigquery==3.34.0 google-cloud-bigquery-storage==2.32.0 google-cloud-bigtable==2.31.0 google-cloud-core==2.4.3 google-cloud-datastore==2.21.0 -google-cloud-dlp==3.30.0 +google-cloud-dlp==3.31.0 google-cloud-language==2.17.2 google-cloud-profiler==4.1.0 google-cloud-pubsub==2.30.0 @@ -87,7 +79,7 @@ google-cloud-storage==2.19.0 google-cloud-videointelligence==2.16.2 google-cloud-vision==3.10.2 google-crc32c==1.7.1 -google-genai==1.20.0 +google-genai==1.22.0 google-resumable-media==2.7.2 googleapis-common-protos==1.70.0 greenlet==3.2.3 @@ -101,31 +93,27 @@ hdfs==2.7.3 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -<<<<<<< HEAD -hypothesis==6.135.7 -======= -hypothesis==6.135.10 ->>>>>>> upstream/master +hypothesis==6.135.16 idna==3.10 importlib_metadata==8.7.0 iniconfig==2.1.0 jaraco.classes==3.4.0 jaraco.context==6.0.1 -jaraco.functools==4.1.0 +jaraco.functools==4.2.1 jeepney==0.9.0 Jinja2==3.1.6 joblib==1.5.1 jsonpickle==3.4.2 jsonschema==4.24.0 jsonschema-specifications==2025.4.1 -kafka-python==2.2.11 +kafka-python==2.2.14 keyring==25.6.0 keyrings.google-artifactregistry-auth==1.1.2 MarkupSafe==3.0.2 mmh3==5.1.0 mock==5.2.0 more-itertools==10.7.0 -multidict==6.4.4 +multidict==6.6.0 mysql-connector-python==9.3.0 nltk==3.9.1 numpy==2.2.6 @@ -134,7 +122,7 @@ objsize==0.7.1 opentelemetry-api==1.34.1 opentelemetry-sdk==1.34.1 opentelemetry-semantic-conventions==0.55b1 -oracledb==3.1.1 +oracledb==3.2.0 orjson==3.10.18 overrides==7.7.0 packaging==25.0 @@ -156,7 +144,7 @@ pydantic_core==2.33.2 pydot==1.4.2 PyHamcrest==2.1.0 PyJWT==2.9.0 -pymongo==4.13.1 +pymongo==4.13.2 PyMySQL==1.1.1 pyparsing==3.2.3 pyproject_hooks==1.2.0 @@ -195,7 +183,7 @@ typing-inspection==0.4.1 typing_extensions==4.14.0 tzdata==2025.2 uritemplate==4.2.0 -urllib3==2.4.0 +urllib3==2.5.0 virtualenv-clone==0.5.7 websockets==15.0.1 wrapt==1.17.2 diff --git a/sdks/python/container/py311/base_image_requirements.txt b/sdks/python/container/py311/base_image_requirements.txt index 966e55c95414..f6e526d174d4 100644 --- a/sdks/python/container/py311/base_image_requirements.txt +++ b/sdks/python/container/py311/base_image_requirements.txt @@ -54,22 +54,18 @@ fasteners==0.19 freezegun==1.5.2 frozenlist==1.7.0 future==1.0.0 -<<<<<<< HEAD -google-api-core==2.25.0 -======= google-api-core==2.25.1 ->>>>>>> upstream/master -google-api-python-client==2.172.0 +google-api-python-client==2.174.0 google-apitools==0.5.31 google-auth==2.40.3 google-auth-httplib2==0.2.0 -google-cloud-aiplatform==1.97.0 +google-cloud-aiplatform==1.100.0 google-cloud-bigquery==3.34.0 google-cloud-bigquery-storage==2.32.0 google-cloud-bigtable==2.31.0 google-cloud-core==2.4.3 google-cloud-datastore==2.21.0 -google-cloud-dlp==3.30.0 +google-cloud-dlp==3.31.0 google-cloud-language==2.17.2 google-cloud-profiler==4.1.0 google-cloud-pubsub==2.30.0 @@ -81,7 +77,7 @@ google-cloud-storage==2.19.0 google-cloud-videointelligence==2.16.2 google-cloud-vision==3.10.2 google-crc32c==1.7.1 -google-genai==1.20.0 +google-genai==1.22.0 google-resumable-media==2.7.2 googleapis-common-protos==1.70.0 greenlet==3.2.3 @@ -95,31 +91,27 @@ hdfs==2.7.3 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -<<<<<<< HEAD -hypothesis==6.135.7 -======= -hypothesis==6.135.10 ->>>>>>> upstream/master +hypothesis==6.135.16 idna==3.10 importlib_metadata==8.7.0 iniconfig==2.1.0 jaraco.classes==3.4.0 jaraco.context==6.0.1 -jaraco.functools==4.1.0 +jaraco.functools==4.2.1 jeepney==0.9.0 Jinja2==3.1.6 joblib==1.5.1 jsonpickle==3.4.2 jsonschema==4.24.0 jsonschema-specifications==2025.4.1 -kafka-python==2.2.11 +kafka-python==2.2.14 keyring==25.6.0 keyrings.google-artifactregistry-auth==1.1.2 MarkupSafe==3.0.2 mmh3==5.1.0 mock==5.2.0 more-itertools==10.7.0 -multidict==6.4.4 +multidict==6.6.0 mysql-connector-python==9.3.0 nltk==3.9.1 numpy==2.2.6 @@ -128,10 +120,7 @@ objsize==0.7.1 opentelemetry-api==1.34.1 opentelemetry-sdk==1.34.1 opentelemetry-semantic-conventions==0.55b1 -<<<<<<< HEAD -======= -oracledb==3.1.1 ->>>>>>> upstream/master +oracledb==3.2.0 orjson==3.10.18 overrides==7.7.0 packaging==25.0 @@ -148,16 +137,12 @@ pyarrow-hotfix==0.7 pyasn1==0.6.1 pyasn1_modules==0.4.2 pycparser==2.22 -<<<<<<< HEAD -pydantic==2.11.5 -======= pydantic==2.11.7 ->>>>>>> upstream/master pydantic_core==2.33.2 pydot==1.4.2 PyHamcrest==2.1.0 PyJWT==2.9.0 -pymongo==4.13.1 +pymongo==4.13.2 PyMySQL==1.1.1 pyparsing==3.2.3 pyproject_hooks==1.2.0 @@ -176,7 +161,7 @@ requests-mock==1.12.1 rpds-py==0.25.1 rsa==4.9.1 scikit-learn==1.7.0 -scipy==1.15.3 +scipy==1.16.0 scramp==1.4.5 SecretStorage==3.3.3 shapely==2.1.1 @@ -195,7 +180,7 @@ typing-inspection==0.4.1 typing_extensions==4.14.0 tzdata==2025.2 uritemplate==4.2.0 -urllib3==2.4.0 +urllib3==2.5.0 virtualenv-clone==0.5.7 websockets==15.0.1 wrapt==1.17.2 diff --git a/sdks/python/container/py312/base_image_requirements.txt b/sdks/python/container/py312/base_image_requirements.txt index 0af67ca373e2..d42229e28ae3 100644 --- a/sdks/python/container/py312/base_image_requirements.txt +++ b/sdks/python/container/py312/base_image_requirements.txt @@ -23,11 +23,7 @@ aiofiles==24.1.0 aiohappyeyeballs==2.6.1 -<<<<<<< HEAD -aiohttp==3.12.12 -======= aiohttp==3.12.13 ->>>>>>> upstream/master aiosignal==1.3.2 annotated-types==0.7.0 anyio==4.9.0 @@ -57,22 +53,18 @@ fasteners==0.19 freezegun==1.5.2 frozenlist==1.7.0 future==1.0.0 -<<<<<<< HEAD -google-api-core==2.25.0 -======= google-api-core==2.25.1 ->>>>>>> upstream/master -google-api-python-client==2.172.0 +google-api-python-client==2.174.0 google-apitools==0.5.31 google-auth==2.40.3 google-auth-httplib2==0.2.0 -google-cloud-aiplatform==1.97.0 +google-cloud-aiplatform==1.100.0 google-cloud-bigquery==3.34.0 google-cloud-bigquery-storage==2.32.0 google-cloud-bigtable==2.31.0 google-cloud-core==2.4.3 google-cloud-datastore==2.21.0 -google-cloud-dlp==3.30.0 +google-cloud-dlp==3.31.0 google-cloud-language==2.17.2 google-cloud-profiler==4.1.0 google-cloud-pubsub==2.30.0 @@ -84,7 +76,7 @@ google-cloud-storage==2.19.0 google-cloud-videointelligence==2.16.2 google-cloud-vision==3.10.2 google-crc32c==1.7.1 -google-genai==1.20.0 +google-genai==1.22.0 google-resumable-media==2.7.2 googleapis-common-protos==1.70.0 greenlet==3.2.3 @@ -98,31 +90,27 @@ hdfs==2.7.3 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -<<<<<<< HEAD -hypothesis==6.135.7 -======= -hypothesis==6.135.10 ->>>>>>> upstream/master +hypothesis==6.135.16 idna==3.10 importlib_metadata==8.7.0 iniconfig==2.1.0 jaraco.classes==3.4.0 jaraco.context==6.0.1 -jaraco.functools==4.1.0 +jaraco.functools==4.2.1 jeepney==0.9.0 Jinja2==3.1.6 joblib==1.5.1 jsonpickle==3.4.2 jsonschema==4.24.0 jsonschema-specifications==2025.4.1 -kafka-python==2.2.11 +kafka-python==2.2.14 keyring==25.6.0 keyrings.google-artifactregistry-auth==1.1.2 MarkupSafe==3.0.2 mmh3==5.1.0 mock==5.2.0 more-itertools==10.7.0 -multidict==6.4.4 +multidict==6.6.0 mysql-connector-python==9.3.0 nltk==3.9.1 numpy==2.2.6 @@ -131,10 +119,7 @@ objsize==0.7.1 opentelemetry-api==1.34.1 opentelemetry-sdk==1.34.1 opentelemetry-semantic-conventions==0.55b1 -<<<<<<< HEAD -======= -oracledb==3.1.1 ->>>>>>> upstream/master +oracledb==3.2.0 orjson==3.10.18 overrides==7.7.0 packaging==25.0 @@ -156,7 +141,7 @@ pydantic_core==2.33.2 pydot==1.4.2 PyHamcrest==2.1.0 PyJWT==2.9.0 -pymongo==4.13.1 +pymongo==4.13.2 PyMySQL==1.1.1 pyparsing==3.2.3 pyproject_hooks==1.2.0 @@ -175,7 +160,7 @@ requests-mock==1.12.1 rpds-py==0.25.1 rsa==4.9.1 scikit-learn==1.7.0 -scipy==1.15.3 +scipy==1.16.0 scramp==1.4.5 SecretStorage==3.3.3 setuptools==80.9.0 @@ -195,7 +180,7 @@ typing-inspection==0.4.1 typing_extensions==4.14.0 tzdata==2025.2 uritemplate==4.2.0 -urllib3==2.4.0 +urllib3==2.5.0 virtualenv-clone==0.5.7 websockets==15.0.1 wheel==0.45.1 diff --git a/sdks/python/container/py39/base_image_requirements.txt b/sdks/python/container/py39/base_image_requirements.txt index c5117ebc741f..b3b176d0c6da 100644 --- a/sdks/python/container/py39/base_image_requirements.txt +++ b/sdks/python/container/py39/base_image_requirements.txt @@ -23,11 +23,7 @@ aiofiles==24.1.0 aiohappyeyeballs==2.6.1 -<<<<<<< HEAD -aiohttp==3.12.12 -======= aiohttp==3.12.13 ->>>>>>> upstream/master aiosignal==1.3.2 annotated-types==0.7.0 anyio==4.9.0 @@ -60,22 +56,18 @@ fasteners==0.19 freezegun==1.5.2 frozenlist==1.7.0 future==1.0.0 -<<<<<<< HEAD -google-api-core==2.25.0 -======= google-api-core==2.25.1 ->>>>>>> upstream/master -google-api-python-client==2.172.0 +google-api-python-client==2.174.0 google-apitools==0.5.31 google-auth==2.40.3 google-auth-httplib2==0.2.0 -google-cloud-aiplatform==1.97.0 +google-cloud-aiplatform==1.100.0 google-cloud-bigquery==3.34.0 google-cloud-bigquery-storage==2.32.0 google-cloud-bigtable==2.31.0 google-cloud-core==2.4.3 google-cloud-datastore==2.21.0 -google-cloud-dlp==3.30.0 +google-cloud-dlp==3.31.0 google-cloud-language==2.17.2 google-cloud-profiler==4.1.0 google-cloud-pubsub==2.30.0 @@ -87,7 +79,7 @@ google-cloud-storage==2.19.0 google-cloud-videointelligence==2.16.2 google-cloud-vision==3.10.2 google-crc32c==1.7.1 -google-genai==1.20.0 +google-genai==1.22.0 google-resumable-media==2.7.2 googleapis-common-protos==1.70.0 greenlet==3.2.3 @@ -101,31 +93,27 @@ hdfs==2.7.3 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -<<<<<<< HEAD -hypothesis==6.135.7 -======= -hypothesis==6.135.10 ->>>>>>> upstream/master +hypothesis==6.135.16 idna==3.10 importlib_metadata==8.7.0 iniconfig==2.1.0 jaraco.classes==3.4.0 jaraco.context==6.0.1 -jaraco.functools==4.1.0 +jaraco.functools==4.2.1 jeepney==0.9.0 Jinja2==3.1.6 joblib==1.5.1 jsonpickle==3.4.2 jsonschema==4.24.0 jsonschema-specifications==2025.4.1 -kafka-python==2.2.11 +kafka-python==2.2.14 keyring==25.6.0 keyrings.google-artifactregistry-auth==1.1.2 MarkupSafe==3.0.2 mmh3==5.1.0 mock==5.2.0 more-itertools==10.7.0 -multidict==6.4.4 +multidict==6.6.0 mysql-connector-python==9.3.0 nltk==3.9.1 numpy==2.0.2 @@ -134,10 +122,7 @@ objsize==0.7.1 opentelemetry-api==1.34.1 opentelemetry-sdk==1.34.1 opentelemetry-semantic-conventions==0.55b1 -<<<<<<< HEAD -======= -oracledb==3.1.1 ->>>>>>> upstream/master +oracledb==3.2.0 orjson==3.10.18 overrides==7.7.0 packaging==25.0 @@ -154,16 +139,12 @@ pyarrow-hotfix==0.7 pyasn1==0.6.1 pyasn1_modules==0.4.2 pycparser==2.22 -<<<<<<< HEAD -pydantic==2.11.5 -======= pydantic==2.11.7 ->>>>>>> upstream/master pydantic_core==2.33.2 pydot==1.4.2 PyHamcrest==2.1.0 PyJWT==2.9.0 -pymongo==4.13.1 +pymongo==4.13.2 PyMySQL==1.1.1 pyparsing==3.2.3 pyproject_hooks==1.2.0 @@ -202,7 +183,7 @@ typing-inspection==0.4.1 typing_extensions==4.14.0 tzdata==2025.2 uritemplate==4.2.0 -urllib3==2.4.0 +urllib3==2.5.0 virtualenv-clone==0.5.7 websockets==15.0.1 wrapt==1.17.2 From 40cdd738895654715de31e4d9fea00eac52ea6af Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Fri, 27 Jun 2025 23:45:16 +0000 Subject: [PATCH 25/83] .github: trigger beam postcommit python sdk --- .github/trigger_files/beam_PostCommit_Python.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/trigger_files/beam_PostCommit_Python.json b/.github/trigger_files/beam_PostCommit_Python.json index 30ee463ad4e9..1eb60f6e4959 100644 --- a/.github/trigger_files/beam_PostCommit_Python.json +++ b/.github/trigger_files/beam_PostCommit_Python.json @@ -1,5 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run.", - "modification": 2 + "modification": 3 } From 8c755bbf198e0514943ca62c36ff37b70b888004 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sat, 28 Jun 2025 14:26:06 +0000 Subject: [PATCH 26/83] examples+transforms: configure sql databases for transform exmaples --- .../transforms/elementwise/enrichment.py | 263 ++++++++++++++---- .../transforms/elementwise/enrichment_test.py | 236 +++++++++++----- .../enrichment_handlers/cloudsql.py | 247 ++++++++-------- .../enrichment_handlers/cloudsql_it_test.py | 111 ++++---- .../enrichment_handlers/cloudsql_test.py | 29 +- 5 files changed, 596 insertions(+), 290 deletions(-) diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py index edfccda75350..b80d1d45ad26 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py @@ -51,59 +51,6 @@ def enrichment_with_bigtable(): # [END enrichment_with_bigtable] -def enrichment_with_cloudsql(): - # [START enrichment_with_cloudsql] - import apache_beam as beam - from apache_beam.transforms.enrichment import Enrichment - from apache_beam.transforms.enrichment_handlers.cloudsql import ( - CloudSQLEnrichmentHandler, - DatabaseTypeAdapter, - TableFieldsQueryConfig, - ExternalSQLDBConnectionConfig) - import os - - database_type_adapter = DatabaseTypeAdapter[os.environ.get("SQL_DB_TYPE")] - database_host = os.environ.get("SQL_DB_HOST") - database_port = int(os.environ.get("SQL_DB_PORT")) - database_user = os.environ.get("SQL_DB_USER") - database_password = os.environ.get("SQL_DB_PASSWORD") - database_id = os.environ.get("SQL_DB_ID") - table_id = "products" - where_clause_template = "product_id = {}" - where_clause_fields = ["product_id"] - - data = [ - beam.Row(product_id=1, name='A'), - beam.Row(product_id=2, name='B'), - beam.Row(product_id=3, name='C'), - ] - - connection_config = ExternalSQLDBConnectionConfig( - db_adapter=database_type_adapter, - host=database_host, - port=database_port, - user=database_user, - password=database_password, - db_id=database_id) - - query_config = TableFieldsQueryConfig( - table_id=table_id, - where_clause_template=where_clause_template, - where_clause_fields=where_clause_fields) - - cloudsql_handler = CloudSQLEnrichmentHandler( - connection_config=connection_config, - table_id=table_id, - query_config=query_config) - with beam.Pipeline() as p: - _ = ( - p - | "Create" >> beam.Create(data) - | "Enrich W/ CloudSQL" >> Enrichment(cloudsql_handler) - | "Print" >> beam.Map(print)) - # [END enrichment_with_cloudsql] - - def enrichment_with_vertex_ai(): # [START enrichment_with_vertex_ai] import apache_beam as beam @@ -169,3 +116,213 @@ def enrichment_with_vertex_ai_legacy(): | "Enrich W/ Vertex AI" >> Enrichment(vertex_ai_handler) | "Print" >> beam.Map(print)) # [END enrichment_with_vertex_ai_legacy] + + +def enrichment_with_google_cloudsql_pg(): + # [START enrichment_with_google_cloudsql_pg] + import apache_beam as beam + from apache_beam.transforms.enrichment import Enrichment + from apache_beam.transforms.enrichment_handlers.cloudsql import ( + CloudSQLEnrichmentHandler, + DatabaseTypeAdapter, + TableFieldsQueryConfig, + CloudSQLConnectionConfig) + import os + + database_adapter = DatabaseTypeAdapter.POSTGRESQL + database_uri = os.environ.get("GOOGLE_CLOUD_SQL_DB_URI") + database_user = int(os.environ.get("GOOGLE_CLOUD_SQL_DB_USER")) + database_password = os.environ.get("GOOGLE_CLOUD_SQL_DB_PASSWORD") + database_id = os.environ.get("GOOGLE_CLOUD_SQL_DB_ID") + table_id = os.environ.get("GOOGLE_CLOUD_SQL_DB_TABLE_ID") + where_clause_template = "product_id = {}" + where_clause_fields = ["product_id"] + + data = [ + beam.Row(product_id=1, name='A'), + beam.Row(product_id=2, name='B'), + beam.Row(product_id=3, name='C'), + ] + + connection_config = CloudSQLConnectionConfig( + db_adapter=database_adapter, + instance_connection_uri=database_uri, + user=database_user, + password=database_password, + db_id=database_id) + + query_config = TableFieldsQueryConfig( + table_id=table_id, + where_clause_template=where_clause_template, + where_clause_fields=where_clause_fields) + + cloudsql_handler = CloudSQLEnrichmentHandler( + connection_config=connection_config, + table_id=table_id, + query_config=query_config) + with beam.Pipeline() as p: + _ = ( + p + | "Create" >> beam.Create(data) + | "Enrich W/ Google CloudSQL PostgreSQL" >> Enrichment(cloudsql_handler) + | "Print" >> beam.Map(print)) + # [END enrichment_with_google_cloudsql_pg] + + +def enrichment_with_external_pg(): + # [START enrichment_with_external_pg] + import apache_beam as beam + from apache_beam.transforms.enrichment import Enrichment + from apache_beam.transforms.enrichment_handlers.cloudsql import ( + CloudSQLEnrichmentHandler, + DatabaseTypeAdapter, + TableFieldsQueryConfig, + ExternalSQLDBConnectionConfig) + import os + + database_adapter = DatabaseTypeAdapter.POSTGRESQL + database_host = os.environ.get("EXTERNAL_SQL_DB_HOST") + database_port = int(os.environ.get("EXTERNAL_SQL_DB_PORT")) + database_user = os.environ.get("EXTERNAL_SQL_DB_USER") + database_password = os.environ.get("EXTERNAL_SQL_DB_PASSWORD") + database_id = os.environ.get("EXTERNAL_SQL_DB_ID") + table_id = os.environ.get("EXTERNAL_SQL_DB_TABLE_ID") + where_clause_template = "product_id = {}" + where_clause_fields = ["product_id"] + + data = [ + beam.Row(product_id=1, name='A'), + beam.Row(product_id=2, name='B'), + beam.Row(product_id=3, name='C'), + ] + + connection_config = ExternalSQLDBConnectionConfig( + db_adapter=database_adapter, + host=database_host, + port=database_port, + user=database_user, + password=database_password, + db_id=database_id) + + query_config = TableFieldsQueryConfig( + table_id=table_id, + where_clause_template=where_clause_template, + where_clause_fields=where_clause_fields) + + cloudsql_handler = CloudSQLEnrichmentHandler( + connection_config=connection_config, + table_id=table_id, + query_config=query_config) + with beam.Pipeline() as p: + _ = ( + p + | "Create" >> beam.Create(data) + | "Enrich W/ Unmanaged PostgreSQL" >> Enrichment(cloudsql_handler) + | "Print" >> beam.Map(print)) + # [END enrichment_with_external_pg] + + +def enrichment_with_external_mysql(): + # [START enrichment_with_external_mysql] + import apache_beam as beam + from apache_beam.transforms.enrichment import Enrichment + from apache_beam.transforms.enrichment_handlers.cloudsql import ( + CloudSQLEnrichmentHandler, + DatabaseTypeAdapter, + TableFieldsQueryConfig, + ExternalSQLDBConnectionConfig) + import os + + database_adapter = DatabaseTypeAdapter.MYSQL + database_host = os.environ.get("EXTERNAL_SQL_DB_HOST") + database_port = int(os.environ.get("EXTERNAL_SQL_DB_PORT")) + database_user = os.environ.get("EXTERNAL_SQL_DB_USER") + database_password = os.environ.get("EXTERNAL_SQL_DB_PASSWORD") + database_id = os.environ.get("EXTERNAL_SQL_DB_ID") + table_id = os.environ.get("EXTERNAL_SQL_DB_TABLE_ID") + where_clause_template = "product_id = {}" + where_clause_fields = ["product_id"] + + data = [ + beam.Row(product_id=1, name='A'), + beam.Row(product_id=2, name='B'), + beam.Row(product_id=3, name='C'), + ] + + connection_config = ExternalSQLDBConnectionConfig( + db_adapter=database_adapter, + host=database_host, + port=database_port, + user=database_user, + password=database_password, + db_id=database_id) + + query_config = TableFieldsQueryConfig( + table_id=table_id, + where_clause_template=where_clause_template, + where_clause_fields=where_clause_fields) + + cloudsql_handler = CloudSQLEnrichmentHandler( + connection_config=connection_config, + table_id=table_id, + query_config=query_config) + with beam.Pipeline() as p: + _ = ( + p + | "Create" >> beam.Create(data) + | "Enrich W/ Unmanaged MySQL" >> Enrichment(cloudsql_handler) + | "Print" >> beam.Map(print)) + # [END enrichment_with_external_mysql] + + +def enrichment_with_external_sqlserver(): + # [START enrichment_with_external_sqlserver] + import apache_beam as beam + from apache_beam.transforms.enrichment import Enrichment + from apache_beam.transforms.enrichment_handlers.cloudsql import ( + CloudSQLEnrichmentHandler, + DatabaseTypeAdapter, + TableFieldsQueryConfig, + ExternalSQLDBConnectionConfig) + import os + + database_adapter = DatabaseTypeAdapter.SQLSERVER + database_host = os.environ.get("EXTERNAL_SQL_DB_HOST") + database_port = int(os.environ.get("EXTERNAL_SQL_DB_PORT")) + database_user = os.environ.get("EXTERNAL_SQL_DB_USER") + database_password = os.environ.get("EXTERNAL_SQL_DB_PASSWORD") + database_id = os.environ.get("EXTERNAL_SQL_DB_ID") + table_id = os.environ.get("EXTERNAL_SQL_DB_TABLE_ID") + where_clause_template = "product_id = {}" + where_clause_fields = ["product_id"] + + data = [ + beam.Row(product_id=1, name='A'), + beam.Row(product_id=2, name='B'), + beam.Row(product_id=3, name='C'), + ] + + connection_config = ExternalSQLDBConnectionConfig( + db_adapter=database_adapter, + host=database_host, + port=database_port, + user=database_user, + password=database_password, + db_id=database_id) + + query_config = TableFieldsQueryConfig( + table_id=table_id, + where_clause_template=where_clause_template, + where_clause_fields=where_clause_fields) + + cloudsql_handler = CloudSQLEnrichmentHandler( + connection_config=connection_config, + table_id=table_id, + query_config=query_config) + with beam.Pipeline() as p: + _ = ( + p + | "Create" >> beam.Create(data) + | "Enrich W/ Unmanaged SQLServer" >> Enrichment(cloudsql_handler) + | "Print" >> beam.Map(print)) + # [END enrichment_with_external_sqlserver] diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py index 36b46cb5a74e..2e01bc85046f 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py @@ -18,26 +18,35 @@ # pytype: skip-file # pylint: disable=line-too-long +from contextlib import contextmanager import os +from typing import Optional import unittest from io import StringIO -from typing import Tuple +from dataclasses import dataclass import mock import pytest # pylint: disable=unused-import try: - from sqlalchemy import Column, Integer, String, Engine, MetaData + from sqlalchemy import ( + Column, Integer, VARCHAR, Engine, MetaData, create_engine) from apache_beam.examples.snippets.transforms.elementwise.enrichment import ( enrichment_with_bigtable, enrichment_with_vertex_ai_legacy) from apache_beam.examples.snippets.transforms.elementwise.enrichment import ( - enrichment_with_vertex_ai, enrichment_with_cloudsql) + enrichment_with_vertex_ai, + enrichment_with_google_cloudsql_pg, + enrichment_with_external_pg, + enrichment_with_external_mysql, + enrichment_with_external_sqlserver) from apache_beam.transforms.enrichment_handlers.cloudsql import ( DatabaseTypeAdapter) from apache_beam.transforms.enrichment_handlers.cloudsql_it_test import ( CloudSQLEnrichmentTestHelper, SQLDBContainerInfo, + ConnectionConfig, + CloudSQLConnectionConfig, ExternalSQLDBConnectionConfig, SQLClientConnectionHandler) from apache_beam.io.requestresponse import RequestResponseIO @@ -54,15 +63,6 @@ def validate_enrichment_with_bigtable(): return expected -def validate_enrichment_with_cloudsql(): - expected = '''[START enrichment_with_cloudsql] -Row(product_id=1, name='A', quantity=2, region_id=3) -Row(product_id=2, name='B', quantity=3, region_id=1) -Row(product_id=3, name='C', quantity=10, region_id=4) - [END enrichment_with_cloudsql]'''.splitlines()[1:-1] - return expected - - def validate_enrichment_with_vertex_ai(): expected = '''[START enrichment_with_vertex_ai] Row(user_id='2963', product_id=14235, sale_price=15.0, age=12.0, state='1', gender='1', country='1') @@ -81,6 +81,15 @@ def validate_enrichment_with_vertex_ai_legacy(): return expected +def validate_enrichment_with_sql(): + expected = '''[START enrichment_with_sql] +Row(product_id=1, name='A', quantity=2, region_id=3) +Row(product_id=2, name='B', quantity=3, region_id=1) +Row(product_id=3, name='C', quantity=10, region_id=4) + [END enrichment_with_sql]'''.splitlines()[1:-1] + return expected + + @mock.patch('sys.stdout', new_callable=StringIO) @pytest.mark.uses_testcontainer class EnrichmentTest(unittest.TestCase): @@ -90,24 +99,6 @@ def test_enrichment_with_bigtable(self, mock_stdout): expected = validate_enrichment_with_bigtable() self.assertEqual(output, expected) - def test_enrichment_with_cloudsql(self, mock_stdout): - db, engine = None, None - try: - db, handler, metadata, engine = self.pre_cloudsql_enrichment_test() - enrichment_with_cloudsql() - output = mock_stdout.getvalue().splitlines() - expected = validate_enrichment_with_cloudsql() - self.assertEqual(output, expected) - except Exception as e: - self.fail(f"Test failed with unexpected error: {e}") - finally: - if db and engine: - self.post_cloudsql_enrichment_test( - db=db, - sql_client_connection_handler=handler, - metadata=metadata, - engine=engine) - def test_enrichment_with_vertex_ai(self, mock_stdout): enrichment_with_vertex_ai() output = mock_stdout.getvalue().splitlines() @@ -123,11 +114,78 @@ def test_enrichment_with_vertex_ai_legacy(self, mock_stdout): self.maxDiff = None self.assertEqual(output, expected) - def pre_cloudsql_enrichment_test(self): - table_id = "products" + def test_enrichment_with_google_cloudsql_pg(self, mock_stdout): + db_adapter = DatabaseTypeAdapter.POSTGRESQL + with EnrichmentTestHelpers.sql_test_context(True, db_adapter): + try: + enrichment_with_google_cloudsql_pg() + output = mock_stdout.getvalue().splitlines() + expected = validate_enrichment_with_sql() + self.assertEqual(output, expected) + except Exception as e: + self.fail(f"Test failed with unexpected error: {e}") + + def test_enrichment_with_external_pg(self, mock_stdout): + db_adapter = DatabaseTypeAdapter.POSTGRESQL + with EnrichmentTestHelpers.sql_test_context(False, db_adapter): + try: + enrichment_with_external_pg() + output = mock_stdout.getvalue().splitlines() + expected = validate_enrichment_with_sql() + self.assertEqual(output, expected) + except Exception as e: + self.fail(f"Test failed with unexpected error: {e}") + + def test_enrichment_with_external_mysql(self, mock_stdout): + db_adapter = DatabaseTypeAdapter.MYSQL + with EnrichmentTestHelpers.sql_test_context(False, db_adapter): + try: + enrichment_with_external_mysql() + output = mock_stdout.getvalue().splitlines() + expected = validate_enrichment_with_sql() + self.assertEqual(output, expected) + except Exception as e: + self.fail(f"Test failed with unexpected error: {e}") + + def test_enrichment_with_external_sqlserver(self, mock_stdout): + db_adapter = DatabaseTypeAdapter.SQLSERVER + with EnrichmentTestHelpers.sql_test_context(False, db_adapter): + try: + enrichment_with_external_sqlserver() + output = mock_stdout.getvalue().splitlines() + expected = validate_enrichment_with_sql() + self.assertEqual(output, expected) + except Exception as e: + self.fail(f"Test failed with unexpected error: {e}") + + +@dataclass +class CloudSQLEnrichmentTestDataConstruct: + client_handler: SQLClientConnectionHandler + engine: Engine + metadata: MetaData + db: SQLDBContainerInfo = None + +class EnrichmentTestHelpers: + @contextmanager + def sql_test_context(is_cloudsql: bool, db_adapter: DatabaseTypeAdapter): + result: Optional[CloudSQLEnrichmentTestDataConstruct] = None + try: + result = EnrichmentTestHelpers.pre_sql_enrichment_test( + is_cloudsql, db_adapter) + yield + finally: + if result: + EnrichmentTestHelpers.post_sql_enrichment_test(result) + + @staticmethod + def pre_sql_enrichment_test( + is_cloudsql: bool, + db_adapter: DatabaseTypeAdapter) -> CloudSQLEnrichmentTestDataConstruct: + table_id = "products_catalog" columns = [ Column("product_id", Integer, primary_key=True), - Column("name", String, nullable=False), + Column("name", VARCHAR(255), nullable=False), Column("quantity", Integer, nullable=False), Column("region_id", Integer, nullable=False), ] @@ -143,51 +201,83 @@ def pre_cloudsql_enrichment_test(self): }, ] metadata = MetaData() - db_adapter = DatabaseTypeAdapter.POSTGRESQL - db = CloudSQLEnrichmentTestHelper.start_sql_db_container(db_adapter) - os.environ['SQL_DB_TYPE'] = db.adapter.name - os.environ['SQL_DB_HOST'] = db.host - os.environ['SQL_DB_PORT'] = str(db.port) - os.environ['SQL_DB_USER'] = db.user - os.environ['SQL_DB_PASSWORD'] = db.password - os.environ['SQL_DB_ID'] = db.id - os.environ['SQL_DB_URL'] = db.url - connection_config = ExternalSQLDBConnectionConfig( - db_adapter=db_adapter, - host=db.host, - port=db.port, - user=db.user, - password=db.password, - db_id=db.id) - handler = CloudSQLEnrichmentTestHelper.create_table( + + connection_config: ConnectionConfig + if is_cloudsql: + gcp_project_id = "apache-beam-testing" + region = "us-central1" + instance_name = "beam-integration-tests" + instance_connection_uri = f"{gcp_project_id}:{region}:{instance_name}" + db_id = "postgres" + user = "postgres" + password = os.getenv("ALLOYDB_PASSWORD") + os.environ['GOOGLE_CLOUD_SQL_DB_URI'] = instance_connection_uri + os.environ['GOOGLE_CLOUD_SQL_DB_ID'] = db_id + os.environ['GOOGLE_CLOUD_SQL_DB_USER'] = user + os.environ['GOOGLE_CLOUD_SQL_DB_PASSWORD'] = password + os.environ['GOOGLE_CLOUD_SQL_DB_TABLE_ID'] = table_id + connection_config = CloudSQLConnectionConfig( + db_adapter=db_adapter, + instance_connection_uri=instance_connection_uri, + user=user, + password=password, + db_id=db_id) + else: + db = CloudSQLEnrichmentTestHelper.start_sql_db_container(db_adapter) + os.environ['EXTERNAL_SQL_DB_HOST'] = db.host + os.environ['EXTERNAL_SQL_DB_PORT'] = str(db.port) + os.environ['EXTERNAL_SQL_DB_ID'] = db.id + os.environ['EXTERNAL_SQL_DB_USER'] = db.user + os.environ['EXTERNAL_SQL_DB_PASSWORD'] = db.password + os.environ['EXTERNAL_SQL_DB_TABLE_ID'] = table_id + connection_config = ExternalSQLDBConnectionConfig( + db_adapter=db_adapter, + host=db.host, + port=db.port, + user=db.user, + password=db.password, + db_id=db.id) + + sql_client_handler = connection_config.get_connector_handler() + engine = create_engine( + url=connection_config.get_db_url(), + creator=sql_client_handler.connector) + + CloudSQLEnrichmentTestHelper.create_table( table_id=table_id, - connection_config=connection_config, + engine=engine, columns=columns, table_data=table_data, metadata=metadata) - handler: Tuple[SQLClientConnectionHandler, Engine] - - sql_client_connection_handler, engine = handler - return db, sql_client_connection_handler, metadata, engine - - def post_cloudsql_enrichment_test( - self, - db: SQLDBContainerInfo, - sql_client_connection_handler: SQLClientConnectionHandler, - metadata: MetaData, - engine: Engine): - metadata.drop_all(engine) - sql_client_connection_handler.connection_closer() - engine.dispose(close=True) - CloudSQLEnrichmentTestHelper.stop_sql_db_container(db) - os.environ.pop('SQL_DB_TYPE', None) - os.environ.pop('SQL_DB_HOST', None) - os.environ.pop('SQL_DB_PORT', None) - os.environ.pop('SQL_DB_USER', None) - os.environ.pop('SQL_DB_PASSWORD', None) - os.environ.pop('SQL_DB_ID', None) - os.environ.pop('SQL_DB_URL', None) + result = CloudSQLEnrichmentTestDataConstruct( + db=db, + client_handler=sql_client_handler, + engine=engine, + metadata=metadata) + return result + + @staticmethod + def post_sql_enrichment_test(res: CloudSQLEnrichmentTestDataConstruct): + # Clean up the data inserted previously. + res.metadata.drop_all(res.engine) + res.engine.dispose(close=True) + + # Check if the test used a container-based external SQL database. + if res.db: + CloudSQLEnrichmentTestHelper.stop_sql_db_container(res.db) + os.environ.pop('EXTERNAL_SQL_DB_HOST', None) + os.environ.pop('EXTERNAL_SQL_DB_PORT', None) + os.environ.pop('EXTERNAL_SQL_DB_ID', None) + os.environ.pop('EXTERNAL_SQL_DB_USER', None) + os.environ.pop('EXTERNAL_SQL_DB_PASSWORD', None) + os.environ.pop('EXTERNAL_SQL_DB_TABLE_ID', None) + else: + os.environ.pop('GOOGLE_CLOUD_SQL_DB_URI', None) + os.environ.pop('GOOGLE_CLOUD_SQL_DB_ID', None) + os.environ.pop('GOOGLE_CLOUD_SQL_DB_USER', None) + os.environ.pop('GOOGLE_CLOUD_SQL_DB_PASSWORD', None) + os.environ.pop('GOOGLE_CLOUD_SQL_DB_TABLE_ID', None) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py index 1f62b052cddd..335209fbd78e 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py @@ -17,9 +17,9 @@ from abc import abstractmethod, ABC from collections.abc import Callable from collections.abc import Mapping -from dataclasses import dataclass +from dataclasses import dataclass, field from enum import Enum -from typing import Any +from typing import Any, Dict, Set from typing import List from typing import Optional from typing import Union @@ -32,7 +32,6 @@ from sqlalchemy.engine import Connection as DBAPIConnection from google.cloud.sql.connector.enums import RefreshStrategy from google.cloud.sql.connector import Connector as CloudSQLConnector -from google.cloud.sql.connector import IPTypes import apache_beam as beam from apache_beam.transforms.enrichment import EnrichmentSourceHandler @@ -127,117 +126,136 @@ def get_db_url(self) -> str: pass +@dataclass class CloudSQLConnectionConfig(ConnectionConfig): - """Connects to Google Cloud SQL using Cloud SQL Python Connector.""" - SUPPORTED_ADAPTERS = { - DatabaseTypeAdapter.POSTGRESQL, - DatabaseTypeAdapter.MYSQL, - DatabaseTypeAdapter.SQLSERVER, - } - - def __init__( - self, - db_adapter: DatabaseTypeAdapter, - instance_connection_uri: str, - user: str, - db_id: str, - enable_iam_auth: bool = True, - password: Optional[str] = None, # fallback if IAM not used - ip_type: IPTypes = IPTypes.PUBLIC, - refresh_strategy: RefreshStrategy = RefreshStrategy.LAZY, - *, - connector_kwargs: Optional[dict] = None, - connect_kwargs: Optional[dict] = None, - ): - self._validate_metadata(db_adapter=db_adapter) - self._db_adapter = db_adapter - self._instance_connection_uri = instance_connection_uri - self._user = user - self._db_id = db_id - self._enable_iam_auth = enable_iam_auth - self._password = password - self._ip_type = ip_type - self._refresh_strategy = refresh_strategy - self.connector_kwargs = connector_kwargs or {} - self.connect_kwargs = connect_kwargs or {} + """Connects to Google Cloud SQL using Cloud SQL Python Connector. - def get_connector_handler(self) -> SQLClientConnectionHandler: - cloudsql_client = CloudSQLConnector( - ip_type=self._ip_type, - refresh_strategy=self._refresh_strategy, - **self.connector_kwargs) + Args: + db_adapter: The database adapter type (PostgreSQL, MySQL, SQL Server). + instance_connection_uri: URI for connecting to the Cloud SQL instance. + user: Username for authentication. + password: Password for authentication. Defaults to None. + db_id: Database identifier/name. + refresh_strategy: Strategy for refreshing connection (default: LAZY). + connector_kwargs: Additional keyword arguments for the + Cloud SQL Python Connector. Enables forward compatibility. + connect_kwargs: Additional keyword arguments for the client connect + method. Enables forward compatibility. + """ + db_adapter: DatabaseTypeAdapter + instance_connection_uri: str + user: str = field(default_factory=str) + password: str = field(default_factory=str) + db_id: str = field(default_factory=str) + refresh_strategy: RefreshStrategy = RefreshStrategy.LAZY + connector_kwargs: Dict[str, Any] = field(default_factory=dict) + connect_kwargs: Dict[str, Any] = field(default_factory=dict) - cloudsql_connector = lambda: cloudsql_client.connect( - instance_connection_string=self._instance_connection_uri, driver=self. - _db_adapter.value, user=self._user, db=self._db_id, enable_iam_auth=self - ._enable_iam_auth, password=self.password, **self.connect_kwargs) + def __post_init__(self): + if not self.instance_connection_uri: + raise ValueError("Instance connection URI cannot be empty") - connection_closer = lambda: cloudsql_client.close() + def get_connector_handler(self) -> SQLClientConnectionHandler: + cloudsql_client = CloudSQLConnector( + refresh_strategy=self.refresh_strategy, + **self.connector_kwargs) - return SQLClientConnectionHandler( - connector=cloudsql_connector, connection_closer=connection_closer) + cloudsql_connector = lambda: cloudsql_client.connect( + instance_connection_string=self.instance_connection_uri, + driver=self.db_adapter.value, + user=self.user, + password=self.password, + db=self.db_id, + **self.connect_kwargs) - def get_db_url(self) -> str: - return self._db_adapter.to_sqlalchemy_dialect() + "://" + connection_closer = lambda: cloudsql_client.close() - def _validate_metadata(self, db_adapter: DatabaseTypeAdapter): - if db_adapter not in self.SUPPORTED_ADAPTERS: - raise ValueError( - f"Unsupported DB adapter for CloudSQLConnectionConfig: {db_adapter}. " - f"Supported: {[a.name for a in self.SUPPORTED_ADAPTERS]}") + return SQLClientConnectionHandler( + connector=cloudsql_connector, connection_closer=connection_closer) - @property - def password(self): - return self._password if not self._enable_iam_auth else None + def get_db_url(self) -> str: + return self.db_adapter.to_sqlalchemy_dialect() + "://" +@dataclass class ExternalSQLDBConnectionConfig(ConnectionConfig): - """Connects to External SQL databases (PostgreSQL, MySQL, etc.) over TCP.""" - def __init__( - self, - db_adapter: DatabaseTypeAdapter, - host: str, - user: str, - password: str, - db_id: str, - port: int, - **kwargs): - self._db_adapter = db_adapter - self._host = host - self._user = user - self._password = password - self._db_id = db_id - self._port = port - self.kwargs = kwargs + """Connects to External SQL DBs (PostgreSQL, MySQL, SQL Server) over TCP. - def get_connector_handler( - self, - ) -> SQLClientConnectionHandler: - if self._db_adapter == DatabaseTypeAdapter.POSTGRESQL: - # It is automatically closed upstream by sqlalchemy. - connector = lambda: pg8000.connect( - host=self._host, user=self._user, password=self._password, database= - self._db_id, port=self._port, **self.kwargs) - connection_closer = lambda: None - elif self._db_adapter == DatabaseTypeAdapter.MYSQL: - # It is automatically closed upstream by sqlalchemy. - connector = lambda: pymysql.connect( - host=self._host, user=self._user, password=self._password, database= - self._db_id, port=self._port, **self.kwargs) - connection_closer = lambda: None - elif self._db_adapter == DatabaseTypeAdapter.SQLSERVER: - # It is automatically closed upstream by sqlalchemy. - connector = lambda: pytds.connect( - server=self._host, database=self._db_id, user=self._user, password= - self._password, port=self._port, **self.kwargs) - connection_closer = lambda: None - else: - raise ValueError(f"Unsupported DB adapter: {self._db_adapter}") - - return SQLClientConnectionHandler(connector, connection_closer) - - def get_db_url(self) -> str: - return self._db_adapter.to_sqlalchemy_dialect() + "://" + Args: + db_adapter: The database adapter type (PostgreSQL, MySQL, SQL Server). + host: Hostname or IP address of the database server. + port: Port number for the database connection. + user: Username for authentication. + password: Password for authentication. + db_id: Database identifier/name. + connect_kwargs: Additional keyword arguments for the client connect + method. Enables forward compatibility. + """ + db_adapter: DatabaseTypeAdapter + host: str + port: int + user: str = field(default_factory=str) + password: str = field(default_factory=str) + db_id: str = field(default_factory=str) + connect_kwargs: Dict[str, Any] = field(default_factory=dict) + + def __post_init__(self): + if not self.host: + raise ValueError("Database host cannot be empty") + + def get_connector_handler(self) -> SQLClientConnectionHandler: + # Use a list to store the connection object because Python closures can + # read but not write to variables in outer scopes. Using a mutable + # object (list) as a container lets the inner functions modify the + # connection state. + connection: List[Optional[DBAPIConnection]]= [None] + if self.db_adapter == DatabaseTypeAdapter.POSTGRESQL: + def connector(): + if connection[0] is None: + connection[0] = pg8000.connect( + host=self.host, + user=self.user, + password=self.password, + database=self.db_id, + port=self.port, + **self.connect_kwargs) + return connection[0] + elif self.db_adapter == DatabaseTypeAdapter.MYSQL: + def connector(): + if connection[0] is None: + connection[0] = pymysql.connect( + host=self.host, + user=self.user, + password=self.password, + database=self.db_id, + port=self.port, + **self.connect_kwargs) + return connection[0] + elif self.db_adapter == DatabaseTypeAdapter.SQLSERVER: + def connector(): + if connection[0] is None: + connection[0] = pytds.connect( + dsn=self.host, + database=self.db_id, + user=self.user, + password=self.password, + port=self.port, + **self.connect_kwargs) + return connection[0] + + # Unified connection closer for all database adapters. + def connection_closer(): + if connection[0]: + try: + connection[0].close() + except Exception as e: + raise ConnectionError( + f"Failed to close {self.db_adapter} connection: {e}") + + return SQLClientConnectionHandler(connector, connection_closer) + + def get_db_url(self) -> str: + return self.db_adapter.to_sqlalchemy_dialect() + "://" QueryConfig = Union[CustomQueryConfig, @@ -279,11 +297,17 @@ def __init__( ): """ Example Usage: - handler = CloudSQLEnrichmentHandler( - connection_config=CloudSQLConnectionConfig(...), - query_config=TableFieldsQueryConfig('my_table',"id = '{}'",['id']), - min_batch_size=2, - max_batch_size=100) + connection_config = CloudSQLConnectionConfig( + db_adapter=DatabaseTypeAdapter.POSTGRESQL, + instance_connection_uri="apache-beam-testing:us-central1:itests", + user=postgres, + password= os.getenv("CLOUDSQL_PG_PASSWORD")) + query_config=TableFieldsQueryConfig('my_table',"id = '{}'",['id']), + cloudsql_handler = CloudSQLEnrichmentHandler( + connection_config=connection_config, + query_config=query_config, + min_batch_size=2, + max_batch_size=100) Args: connection_config (ConnectionConfig): Configuration for connecting to @@ -330,11 +354,12 @@ def __enter__(self): self._engine = create_engine( url=self._connection_config.get_db_url(), creator=self._sql_client_handler.connector) - self._connection = self._engine.connect() def _execute_query(self, query: str, is_batch: bool, **params): try: - result = self._connection.execute(text(query), **params) + with self._engine.connect() as connection: + result = connection.execute(text(query), **params) + connection.commit() if is_batch: return [row._asdict() for row in result] else: @@ -434,10 +459,8 @@ def create_row_key(self, row: beam.Row): "Either where_clause_fields or where_clause_value_fn must be specified") def __exit__(self, exc_type, exc_val, exc_tb): - self._connection.close() - self._sql_client_handler.connection_closer() self._engine.dispose(close=True) - self._engine, self._connection = None, None + self._engine = None def get_cache_key(self, request: Union[beam.Row, list[beam.Row]]): if isinstance(self._query_config, CustomQueryConfig): diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py index bf08a82bcba5..2adeadfdce50 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py @@ -33,7 +33,6 @@ # pylint: disable=ungrouped-imports try: - from google.cloud.sql.connector import IPTypes from testcontainers.core.generic import DbContainer from testcontainers.postgres import PostgresContainer from testcontainers.mysql import MySqlContainer @@ -52,8 +51,8 @@ ExternalSQLDBConnectionConfig, ConnectionConfig, SQLClientConnectionHandler) -except ImportError: - raise unittest.SkipTest('Google Cloud SQL dependencies are not installed.') +except ImportError as e: + raise unittest.SkipTest(f'CloudSQL dependencies not installed: {str(e)}') _LOGGER = logging.getLogger(__name__) @@ -145,18 +144,27 @@ def start_sql_db_container( info.address) break except Exception as e: + stdout_logs, stderr_logs = sql_db_container.get_logs() + stdout_logs = stdout_logs.decode("utf-8") + stderr_logs = stderr_logs.decode("utf-8") _LOGGER.warning( - "Retry %d/%d: Failed to start %s container. Reason: %s", + "Retry %d/%d: Failed to start %s container. Reason: %s. " + "STDOUT logs:\n%s\nSTDERR logs:\n%s", i + 1, sql_client_retries, database_type.name, - e) + e, + stdout_logs, + stderr_logs) if i == sql_client_retries - 1: _LOGGER.error( "Unable to start %s container for I/O tests after %d " - "retries. Tests cannot proceed.", + "retries. Tests cannot proceed. STDOUT logs:\n%s\n" + "STDERR logs:\n%s", database_type.name, - sql_client_retries) + sql_client_retries, + stdout_logs, + stderr_logs) raise e return info @@ -176,29 +184,30 @@ def stop_sql_db_container(db_info: SQLDBContainerInfo): @staticmethod def create_table( table_id: str, - connection_config: ConnectionConfig, + engine: Engine, columns: list[Column], table_data: list[dict], - metadata: MetaData) -> Tuple[SQLClientConnectionHandler, Engine]: - sql_client_handler = connection_config.get_connector_handler() - engine = create_engine( - url=connection_config.get_db_url(), - creator=sql_client_handler.connector) + metadata: MetaData): + # Create table metadata. table = Table(table_id, metadata, *columns) - metadata.create_all(engine) - # Insert data into the table. + # Create contextual connection for schema creation. + with engine.connect() as schema_connection: + try: + metadata.create_all(schema_connection) + schema_connection.commit() + except Exception as e: + schema_connection.rollback() + raise Exception(f"Failed to create table schema: {e}") + + # Now create a separate contextual connection for data insertion. with engine.connect() as connection: - transaction = connection.begin() try: connection.execute(table.insert(), table_data) - transaction.commit() + connection.commit() except Exception as e: - transaction.rollback() - raise e - - return sql_client_handler, engine - + connection.rollback() + raise Exception(f"Failed to insert table data: {e}") @pytest.mark.uses_testcontainer class BaseTestCloudSQLEnrichment(unittest.TestCase): @@ -238,17 +247,21 @@ def setUpClass(cls): # Type hint data from subclasses. cls._table_id: str - cls.connection_config: ConnectionConfig + cls._connection_config: ConnectionConfig cls._metadata: MetaData - handler = CloudSQLEnrichmentTestHelper.create_table( + cls._sql_client_handler = cls._connection_config.get_connector_handler() + cls._engine = create_engine( + url=cls._connection_config.get_db_url(), + creator=cls._sql_client_handler.connector) + + CloudSQLEnrichmentTestHelper.create_table( table_id=cls._table_id, - connection_config=cls.connection_config, + engine=cls._engine, columns=cls.get_columns(), table_data=cls._table_data, metadata=cls._metadata) - cls._sql_client_handler, cls._engine = handler cls._cache_client_retries = 3 @classmethod @@ -293,6 +306,7 @@ def _start_cache_container(self): @classmethod def tearDownClass(cls): cls._metadata.drop_all(cls._engine) + cls._sql_client_handler.connection_closer() cls._engine.dispose(close=True) cls._engine = None @@ -513,29 +527,17 @@ def setUpClass(cls): # Type hint data from subclasses. cls._db_adapter: DatabaseTypeAdapter - cls._instance_name: str - - # GCP project configuration. - gcp_project_id = "apache-beam-testing" - region = "us-central1" - - # Full instance connection name used for connecting via Cloud SQL. - instance_connection_name = f"{gcp_project_id}:{region}:{cls._instance_uri}" - - # Password auth configuration for CloudSQL instance. - user = "postgres" - password = os.getenv("ALLOYDB_PASSWORD") + cls._instance_connection_uri: str + cls._user: str + cls._password: str + cls._db_id: str - # Database ID for CloudSQL instance. - db_id = "postgres" - - cls.connection_config = CloudSQLConnectionConfig( + cls._connection_config = CloudSQLConnectionConfig( db_adapter=cls._db_adapter, - instance_connection_name=instance_connection_name, - user=user, - password=password, - db_id=db_id, - enable_iam_auth=False) + instance_connection_uri=cls._instance_connection_uri, + user=cls._user, + password=cls._password, + db_id=cls._db_id) super().setUpClass() @classmethod @@ -546,8 +548,19 @@ def tearDownClass(cls): class TestCloudSQLPostgresEnrichment(BaseCloudSQLDBEnrichment): _db_adapter = DatabaseTypeAdapter.POSTGRESQL - _instance_name = "beam-integration-tests" + + # Configuration required for locating the CloudSQL instance. _table_id = "product_details_cloudsql_pg_enrichment" + _gcp_project_id = "apache-beam-testing" + _region = "us-central1" + _instance_name = "beam-integration-tests" + _instance_connection_uri = f"{_gcp_project_id}:{_region}:{_instance_name}" + + # Configuration required for authenticating to the CloudSQL instance. + _user = "postgres" + _password = os.getenv("ALLOYDB_PASSWORD") + _db_id = "postgres" + _metadata = MetaData() @@ -564,7 +577,7 @@ def setUpClass(cls): cls._db = CloudSQLEnrichmentTestHelper.start_sql_db_container( cls._db_adapter) - cls.connection_config = ExternalSQLDBConnectionConfig( + cls._connection_config = ExternalSQLDBConnectionConfig( db_adapter=cls._db_adapter, host=cls._db.host, user=cls._db.user, diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_test.py index 6605430bcf27..6cca33b9a3c0 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_test.py @@ -26,16 +26,39 @@ CustomQueryConfig, TableFieldsQueryConfig, TableFunctionQueryConfig, + CloudSQLConnectionConfig, ExternalSQLDBConnectionConfig) from apache_beam.transforms.enrichment_handlers.cloudsql_it_test import ( query_fn, where_clause_value_fn, ) -except ImportError: - raise unittest.SkipTest('Google Cloud SQL dependencies are not installed.') +except ImportError as e: + raise unittest.SkipTest(f'CloudSQL dependencies not installed: {str(e)}') class TestCloudSQLEnrichment(unittest.TestCase): + def test_invalid_external_db_connection_params(self): + with self.assertRaises(ValueError) as context: + _ = ExternalSQLDBConnectionConfig( + db_adapter=DatabaseTypeAdapter.POSTGRESQL, + host='', + port=5432, + user='', + password='', + db_id='') + self.assertIn("Database host cannot be empty", str(context.exception)) + + def test_invalid_cloudsql_db_connection_params(self): + with self.assertRaises(ValueError) as context: + _ = CloudSQLConnectionConfig( + db_adapter=DatabaseTypeAdapter.POSTGRESQL, + instance_connection_uri='', + user='', + password='', + db_id='') + self.assertIn( + "Instance connection URI cannot be empty", str(context.exception)) + @parameterized.expand([ # Empty TableFieldsQueryConfig. ( @@ -106,7 +129,7 @@ def test_invalid_query_config( connection_config = ExternalSQLDBConnectionConfig( db_adapter=DatabaseTypeAdapter.POSTGRESQL, - host='', + host='localhost', port=5432, user='', password='', From 72d8725a35c36245d6a5ec11a80b5b2f4eeb38c1 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sat, 28 Jun 2025 21:50:03 +0000 Subject: [PATCH 27/83] apache_beam: fix dependencies --- .../transforms/elementwise/enrichment_test.py | 18 +-- .../enrichment_handlers/cloudsql.py | 121 +++++++----------- .../enrichment_handlers/cloudsql_it_test.py | 70 +++++----- 3 files changed, 88 insertions(+), 121 deletions(-) diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py index 2e01bc85046f..396bd12e2c9e 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py @@ -41,17 +41,17 @@ enrichment_with_external_mysql, enrichment_with_external_sqlserver) from apache_beam.transforms.enrichment_handlers.cloudsql import ( - DatabaseTypeAdapter) + DatabaseTypeAdapter, + SQLClientConnectionHandler) from apache_beam.transforms.enrichment_handlers.cloudsql_it_test import ( - CloudSQLEnrichmentTestHelper, + SQLEnrichmentTestHelper, SQLDBContainerInfo, ConnectionConfig, CloudSQLConnectionConfig, - ExternalSQLDBConnectionConfig, - SQLClientConnectionHandler) + ExternalSQLDBConnectionConfig) from apache_beam.io.requestresponse import RequestResponseIO -except ImportError: - raise unittest.SkipTest('RequestResponseIO dependencies are not installed') +except ImportError as e: + raise unittest.SkipTest(f'RequestResponseIO dependencies not installed: {e}') def validate_enrichment_with_bigtable(): @@ -223,7 +223,7 @@ def pre_sql_enrichment_test( password=password, db_id=db_id) else: - db = CloudSQLEnrichmentTestHelper.start_sql_db_container(db_adapter) + db = SQLEnrichmentTestHelper.start_sql_db_container(db_adapter) os.environ['EXTERNAL_SQL_DB_HOST'] = db.host os.environ['EXTERNAL_SQL_DB_PORT'] = str(db.port) os.environ['EXTERNAL_SQL_DB_ID'] = db.id @@ -243,7 +243,7 @@ def pre_sql_enrichment_test( url=connection_config.get_db_url(), creator=sql_client_handler.connector) - CloudSQLEnrichmentTestHelper.create_table( + SQLEnrichmentTestHelper.create_table( table_id=table_id, engine=engine, columns=columns, @@ -265,7 +265,7 @@ def post_sql_enrichment_test(res: CloudSQLEnrichmentTestDataConstruct): # Check if the test used a container-based external SQL database. if res.db: - CloudSQLEnrichmentTestHelper.stop_sql_db_container(res.db) + SQLEnrichmentTestHelper.stop_sql_db_container(res.db) os.environ.pop('EXTERNAL_SQL_DB_HOST', None) os.environ.pop('EXTERNAL_SQL_DB_PORT', None) os.environ.pop('EXTERNAL_SQL_DB_ID', None) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py index 335209fbd78e..b3cb03778cab 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py @@ -19,7 +19,7 @@ from collections.abc import Mapping from dataclasses import dataclass, field from enum import Enum -from typing import Any, Dict, Set +from typing import Any, Dict from typing import List from typing import Optional from typing import Union @@ -110,15 +110,9 @@ def to_sqlalchemy_dialect(self): raise ValueError(f"Unsupported database adapter type: {self.name}") -@dataclass -class SQLClientConnectionHandler: - connector: Callable[[], DBAPIConnection] - connection_closer: Callable[[], None] - - class ConnectionConfig(ABC): @abstractmethod - def get_connector_handler(self) -> SQLClientConnectionHandler: + def get_connector_handler(self) -> Callable[[], DBAPIConnection]: pass @abstractmethod @@ -155,7 +149,7 @@ def __post_init__(self): if not self.instance_connection_uri: raise ValueError("Instance connection URI cannot be empty") - def get_connector_handler(self) -> SQLClientConnectionHandler: + def get_connector_handler(self) -> Callable[[], DBAPIConnection]: cloudsql_client = CloudSQLConnector( refresh_strategy=self.refresh_strategy, **self.connector_kwargs) @@ -168,10 +162,7 @@ def get_connector_handler(self) -> SQLClientConnectionHandler: db=self.db_id, **self.connect_kwargs) - connection_closer = lambda: cloudsql_client.close() - - return SQLClientConnectionHandler( - connector=cloudsql_connector, connection_closer=connection_closer) + return cloudsql_connector def get_db_url(self) -> str: return self.db_adapter.to_sqlalchemy_dialect() + "://" @@ -203,56 +194,24 @@ def __post_init__(self): if not self.host: raise ValueError("Database host cannot be empty") - def get_connector_handler(self) -> SQLClientConnectionHandler: - # Use a list to store the connection object because Python closures can - # read but not write to variables in outer scopes. Using a mutable - # object (list) as a container lets the inner functions modify the - # connection state. - connection: List[Optional[DBAPIConnection]]= [None] - if self.db_adapter == DatabaseTypeAdapter.POSTGRESQL: - def connector(): - if connection[0] is None: - connection[0] = pg8000.connect( - host=self.host, - user=self.user, - password=self.password, - database=self.db_id, - port=self.port, - **self.connect_kwargs) - return connection[0] - elif self.db_adapter == DatabaseTypeAdapter.MYSQL: - def connector(): - if connection[0] is None: - connection[0] = pymysql.connect( - host=self.host, - user=self.user, - password=self.password, - database=self.db_id, - port=self.port, - **self.connect_kwargs) - return connection[0] - elif self.db_adapter == DatabaseTypeAdapter.SQLSERVER: - def connector(): - if connection[0] is None: - connection[0] = pytds.connect( - dsn=self.host, - database=self.db_id, - user=self.user, - password=self.password, - port=self.port, - **self.connect_kwargs) - return connection[0] - - # Unified connection closer for all database adapters. - def connection_closer(): - if connection[0]: - try: - connection[0].close() - except Exception as e: - raise ConnectionError( - f"Failed to close {self.db_adapter} connection: {e}") - - return SQLClientConnectionHandler(connector, connection_closer) + def get_connector_handler(self) -> Callable[[], DBAPIConnection]: + if self.db_adapter == DatabaseTypeAdapter.POSTGRESQL: + # It is automatically closed upstream by sqlalchemy context manager. + sql_connector = lambda: pg8000.connect( + host=self.host, port=self.port, database=self.db_id, + user=self.user, password=self.password, **self.connect_kwargs) + elif self.db_adapter == DatabaseTypeAdapter.MYSQL: + # It is automatically closed upstream by sqlalchemy context manager. + sql_connector = lambda: pymysql.connect( + host=self.host, port=self.port, database=self.db_id, + user=self.user, password=self.password, **self.connect_kwargs) + elif self.db_adapter == DatabaseTypeAdapter.SQLSERVER: + # It is automatically closed upstream by sqlalchemy context manager. + sql_connector = lambda: pytds.connect( + dsn=self.host, port=self.port, database=self.db_id, user=self.user, + password=self.password, **self.connect_kwargs) + + return sql_connector def get_db_url(self) -> str: return self.db_adapter.to_sqlalchemy_dialect() + "://" @@ -350,25 +309,35 @@ def __init__( self._batching_kwargs['max_batch_size'] = max_batch_size def __enter__(self): - self._sql_client_handler = self._connection_config.get_connector_handler() + connector = self._connection_config.get_connector_handler() self._engine = create_engine( url=self._connection_config.get_db_url(), - creator=self._sql_client_handler.connector) + creator=connector) def _execute_query(self, query: str, is_batch: bool, **params): try: - with self._engine.connect() as connection: + connection = self._engine.connect() + transaction = connection.begin() + try: result = connection.execute(text(query), **params) - connection.commit() - if is_batch: - return [row._asdict() for row in result] - else: - return result.first()._asdict() - except RuntimeError as e: - raise RuntimeError( - f'Could not execute the query: {query}. Please check if ' - f'the query is properly formatted and the BigQuery ' - f'table exists. {e}') + # Materialize results while transaction is active. + if is_batch: + data = [row._asdict() for row in result] + else: + data = result.first()._asdict() + # Explicitly commit the transaction. + transaction.commit() + return data + except Exception as e: + transaction.rollback() + raise RuntimeError(f"Database operation failed: {e}") + except Exception as e: + raise Exception( + f'Could not execute the query: {query}. Please check if ' + f'the query is properly formatted and the table exists. {e}') + finally: + if connection: + connection.close() def __call__(self, request: Union[beam.Row, list[beam.Row]], *args, **kwargs): """Handle requests by delegating to single or batch processing.""" diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py index 2adeadfdce50..69d489011bcf 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py @@ -49,8 +49,7 @@ TableFunctionQueryConfig, CloudSQLConnectionConfig, ExternalSQLDBConnectionConfig, - ConnectionConfig, - SQLClientConnectionHandler) + ConnectionConfig) except ImportError as e: raise unittest.SkipTest(f'CloudSQL dependencies not installed: {str(e)}') @@ -84,7 +83,7 @@ def url(self) -> str: return self.adapter.to_sqlalchemy_dialect() + "://" -class CloudSQLEnrichmentTestHelper: +class SQLEnrichmentTestHelper: @staticmethod def start_sql_db_container( database_type: DatabaseTypeAdapter, @@ -198,7 +197,7 @@ def create_table( schema_connection.commit() except Exception as e: schema_connection.rollback() - raise Exception(f"Failed to create table schema: {e}") + raise RuntimeError(f"Failed to create table schema: {e}") # Now create a separate contextual connection for data insertion. with engine.connect() as connection: @@ -210,7 +209,7 @@ def create_table( raise Exception(f"Failed to insert table data: {e}") @pytest.mark.uses_testcontainer -class BaseTestCloudSQLEnrichment(unittest.TestCase): +class BaseTestSQLEnrichment(unittest.TestCase): _table_data = [ { "id": 1, "name": "A", 'quantity': 2, 'distribution_center_id': 3 @@ -240,7 +239,7 @@ class BaseTestCloudSQLEnrichment(unittest.TestCase): @classmethod def setUpClass(cls): - if not hasattr(cls, 'connection_config') or not hasattr(cls, '_metadata'): + if not hasattr(cls, '_connection_config') or not hasattr(cls, '_metadata'): # Skip setup for the base class. raise unittest.SkipTest( "Base class - no connection_config or metadata defined") @@ -250,12 +249,12 @@ def setUpClass(cls): cls._connection_config: ConnectionConfig cls._metadata: MetaData - cls._sql_client_handler = cls._connection_config.get_connector_handler() + connector = cls._connection_config.get_connector_handler() cls._engine = create_engine( url=cls._connection_config.get_db_url(), - creator=cls._sql_client_handler.connector) + creator=connector) - CloudSQLEnrichmentTestHelper.create_table( + SQLEnrichmentTestHelper.create_table( table_id=cls._table_id, engine=cls._engine, columns=cls.get_columns(), @@ -306,11 +305,10 @@ def _start_cache_container(self): @classmethod def tearDownClass(cls): cls._metadata.drop_all(cls._engine) - cls._sql_client_handler.connection_closer() cls._engine.dispose(close=True) cls._engine = None - def test_cloudsql_enrichment(self): + def test_sql_enrichment(self): expected_rows = [ beam.Row(id=1, name="A", quantity=2, distribution_center_id=3), beam.Row(id=2, name="B", quantity=3, distribution_center_id=1) @@ -327,17 +325,18 @@ def test_cloudsql_enrichment(self): where_clause_fields=fields) handler = CloudSQLEnrichmentHandler( - connection_config=self.connection_config, + connection_config=self._connection_config, query_config=query_config, min_batch_size=1, max_batch_size=100, ) + with TestPipeline(is_integration_test=True) as test_pipeline: pcoll = (test_pipeline | beam.Create(requests) | Enrichment(handler)) assert_that(pcoll, equal_to(expected_rows)) - def test_cloudsql_enrichment_batched(self): + def test_sql_enrichment_batched(self): expected_rows = [ beam.Row(id=1, name="A", quantity=2, distribution_center_id=3), beam.Row(id=2, name="B", quantity=3, distribution_center_id=1) @@ -354,7 +353,7 @@ def test_cloudsql_enrichment_batched(self): where_clause_fields=fields) handler = CloudSQLEnrichmentHandler( - connection_config=self.connection_config, + connection_config=self._connection_config, query_config=query_config, min_batch_size=2, max_batch_size=100, @@ -364,7 +363,7 @@ def test_cloudsql_enrichment_batched(self): assert_that(pcoll, equal_to(expected_rows)) - def test_cloudsql_enrichment_batched_multiple_fields(self): + def test_sql_enrichment_batched_multiple_fields(self): expected_rows = [ beam.Row(id=1, distribution_center_id=3, name="A", quantity=2), beam.Row(id=2, distribution_center_id=1, name="B", quantity=3) @@ -381,7 +380,7 @@ def test_cloudsql_enrichment_batched_multiple_fields(self): where_clause_fields=fields) handler = CloudSQLEnrichmentHandler( - connection_config=self.connection_config, + connection_config=self._connection_config, query_config=query_config, min_batch_size=8, max_batch_size=100, @@ -391,7 +390,7 @@ def test_cloudsql_enrichment_batched_multiple_fields(self): assert_that(pcoll, equal_to(expected_rows)) - def test_cloudsql_enrichment_with_query_fn(self): + def test_sql_enrichment_with_query_fn(self): expected_rows = [ beam.Row(id=1, name="A", quantity=2, distribution_center_id=3), beam.Row(id=2, name="B", quantity=3, distribution_center_id=1) @@ -405,13 +404,13 @@ def test_cloudsql_enrichment_with_query_fn(self): query_config = CustomQueryConfig(query_fn=fn) handler = CloudSQLEnrichmentHandler( - connection_config=self.connection_config, query_config=query_config) + connection_config=self._connection_config, query_config=query_config) with TestPipeline(is_integration_test=True) as test_pipeline: pcoll = (test_pipeline | beam.Create(requests) | Enrichment(handler)) assert_that(pcoll, equal_to(expected_rows)) - def test_cloudsql_enrichment_with_condition_value_fn(self): + def test_sql_enrichment_with_condition_value_fn(self): expected_rows = [ beam.Row(id=1, name="A", quantity=2, distribution_center_id=3), beam.Row(id=2, name="B", quantity=3, distribution_center_id=1) @@ -427,7 +426,7 @@ def test_cloudsql_enrichment_with_condition_value_fn(self): where_clause_value_fn=where_clause_value_fn) handler = CloudSQLEnrichmentHandler( - connection_config=self.connection_config, + connection_config=self._connection_config, query_config=query_config, min_batch_size=2, max_batch_size=100) @@ -436,7 +435,7 @@ def test_cloudsql_enrichment_with_condition_value_fn(self): assert_that(pcoll, equal_to(expected_rows)) - def test_cloudsql_enrichment_table_nonexistent_runtime_error_raised(self): + def test_sql_enrichment_on_non_existent_table(self): requests = [ beam.Row(id=1, name='A'), beam.Row(id=2, name='B'), @@ -448,21 +447,20 @@ def test_cloudsql_enrichment_table_nonexistent_runtime_error_raised(self): where_clause_value_fn=where_clause_value_fn) handler = CloudSQLEnrichmentHandler( - connection_config=self.connection_config, + connection_config=self._connection_config, query_config=query_config, column_names=["wrong_column"], ) - with self.assertRaises(RuntimeError): - test_pipeline = beam.Pipeline() - _ = ( - test_pipeline - | "Create" >> beam.Create(requests) - | "Enrichment" >> Enrichment(handler)) - res = test_pipeline.run() - res.wait_until_finish() + + with self.assertRaises(Exception) as context: + with TestPipeline() as p: + _ = (p | beam.Create(requests) | Enrichment(handler)) + + expect_err_msg_contains = "Could not execute the query" + self.assertIn(expect_err_msg_contains, str(context.exception)) @pytest.mark.usefixtures("cache_container") - def test_cloudsql_enrichment_with_redis(self): + def test_sql_enrichment_with_redis(self): requests = [ beam.Row(id=1, name='A'), beam.Row(id=2, name='B'), @@ -478,7 +476,7 @@ def test_cloudsql_enrichment_with_redis(self): where_clause_value_fn=where_clause_value_fn) handler = CloudSQLEnrichmentHandler( - connection_config=self.connection_config, + connection_config=self._connection_config, query_config=query_config, min_batch_size=2, max_batch_size=100) @@ -518,7 +516,7 @@ def test_cloudsql_enrichment_with_redis(self): CloudSQLEnrichmentHandler.__call__ = actual -class BaseCloudSQLDBEnrichment(BaseTestCloudSQLEnrichment): +class BaseCloudSQLDBEnrichment(BaseTestSQLEnrichment): @classmethod def setUpClass(cls): if not hasattr(cls, '_db_adapter'): @@ -565,7 +563,7 @@ class TestCloudSQLPostgresEnrichment(BaseCloudSQLDBEnrichment): @pytest.mark.uses_testcontainer -class BaseExternalSQLDBEnrichment(BaseTestCloudSQLEnrichment): +class BaseExternalSQLDBEnrichment(BaseTestSQLEnrichment): @classmethod def setUpClass(cls): if not hasattr(cls, '_db_adapter'): @@ -575,7 +573,7 @@ def setUpClass(cls): # Type hint data from subclasses. cls._db_adapter: DatabaseTypeAdapter - cls._db = CloudSQLEnrichmentTestHelper.start_sql_db_container( + cls._db = SQLEnrichmentTestHelper.start_sql_db_container( cls._db_adapter) cls._connection_config = ExternalSQLDBConnectionConfig( db_adapter=cls._db_adapter, @@ -589,7 +587,7 @@ def setUpClass(cls): @classmethod def tearDownClass(cls): super().tearDownClass() - CloudSQLEnrichmentTestHelper.stop_sql_db_container(cls._db) + SQLEnrichmentTestHelper.stop_sql_db_container(cls._db) cls._db = None From 5a4ed41f132394a464e0e8d313ae710a5011dc78 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sat, 28 Jun 2025 21:54:38 +0000 Subject: [PATCH 28/83] sdks/python: update `get_connector_handler` python docstring --- .../enrichment_handlers/cloudsql.py | 36 +++++++++++-------- 1 file changed, 21 insertions(+), 15 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py index b3cb03778cab..bf812b0931c3 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py @@ -150,19 +150,24 @@ def __post_init__(self): raise ValueError("Instance connection URI cannot be empty") def get_connector_handler(self) -> Callable[[], DBAPIConnection]: - cloudsql_client = CloudSQLConnector( - refresh_strategy=self.refresh_strategy, - **self.connector_kwargs) + """Returns a function that creates a new database connection. - cloudsql_connector = lambda: cloudsql_client.connect( - instance_connection_string=self.instance_connection_uri, - driver=self.db_adapter.value, - user=self.user, - password=self.password, - db=self.db_id, - **self.connect_kwargs) + The returned connector function creates database connections that should + be properly closed by the caller when no longer needed. + """ + cloudsql_client = CloudSQLConnector( + refresh_strategy=self.refresh_strategy, + **self.connector_kwargs) - return cloudsql_connector + cloudsql_connector = lambda: cloudsql_client.connect( + instance_connection_string=self.instance_connection_uri, + driver=self.db_adapter.value, + user=self.user, + password=self.password, + db=self.db_id, + **self.connect_kwargs) + + return cloudsql_connector def get_db_url(self) -> str: return self.db_adapter.to_sqlalchemy_dialect() + "://" @@ -195,22 +200,23 @@ def __post_init__(self): raise ValueError("Database host cannot be empty") def get_connector_handler(self) -> Callable[[], DBAPIConnection]: + """Returns a function that creates a new database connection. + + The returned connector function creates database connections that should + be properly closed by the caller when no longer needed. + """ if self.db_adapter == DatabaseTypeAdapter.POSTGRESQL: - # It is automatically closed upstream by sqlalchemy context manager. sql_connector = lambda: pg8000.connect( host=self.host, port=self.port, database=self.db_id, user=self.user, password=self.password, **self.connect_kwargs) elif self.db_adapter == DatabaseTypeAdapter.MYSQL: - # It is automatically closed upstream by sqlalchemy context manager. sql_connector = lambda: pymysql.connect( host=self.host, port=self.port, database=self.db_id, user=self.user, password=self.password, **self.connect_kwargs) elif self.db_adapter == DatabaseTypeAdapter.SQLSERVER: - # It is automatically closed upstream by sqlalchemy context manager. sql_connector = lambda: pytds.connect( dsn=self.host, port=self.port, database=self.db_id, user=self.user, password=self.password, **self.connect_kwargs) - return sql_connector def get_db_url(self) -> str: From e214ce548f9738e8c06acf2d43af5b27970f389a Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sat, 28 Jun 2025 22:02:38 +0000 Subject: [PATCH 29/83] sdks/python: fix linting for cloudsql --- .../transforms/elementwise/enrichment.py | 3 +- .../transforms/elementwise/enrichment_test.py | 74 +++++----- .../enrichment_handlers/cloudsql.py | 137 +++++++++--------- .../enrichment_handlers/cloudsql_it_test.py | 21 ++- .../enrichment_handlers/cloudsql_test.py | 2 +- 5 files changed, 116 insertions(+), 121 deletions(-) diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py index b80d1d45ad26..4df3d2c5d9a6 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py @@ -164,7 +164,8 @@ def enrichment_with_google_cloudsql_pg(): _ = ( p | "Create" >> beam.Create(data) - | "Enrich W/ Google CloudSQL PostgreSQL" >> Enrichment(cloudsql_handler) + | + "Enrich W/ Google CloudSQL PostgreSQL" >> Enrichment(cloudsql_handler) | "Print" >> beam.Map(print)) # [END enrichment_with_google_cloudsql_pg] diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py index 396bd12e2c9e..4138b3e0af2c 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py @@ -41,8 +41,7 @@ enrichment_with_external_mysql, enrichment_with_external_sqlserver) from apache_beam.transforms.enrichment_handlers.cloudsql import ( - DatabaseTypeAdapter, - SQLClientConnectionHandler) + DatabaseTypeAdapter, SQLClientConnectionHandler) from apache_beam.transforms.enrichment_handlers.cloudsql_it_test import ( SQLEnrichmentTestHelper, SQLDBContainerInfo, @@ -117,46 +116,46 @@ def test_enrichment_with_vertex_ai_legacy(self, mock_stdout): def test_enrichment_with_google_cloudsql_pg(self, mock_stdout): db_adapter = DatabaseTypeAdapter.POSTGRESQL with EnrichmentTestHelpers.sql_test_context(True, db_adapter): - try: - enrichment_with_google_cloudsql_pg() - output = mock_stdout.getvalue().splitlines() - expected = validate_enrichment_with_sql() - self.assertEqual(output, expected) - except Exception as e: - self.fail(f"Test failed with unexpected error: {e}") + try: + enrichment_with_google_cloudsql_pg() + output = mock_stdout.getvalue().splitlines() + expected = validate_enrichment_with_sql() + self.assertEqual(output, expected) + except Exception as e: + self.fail(f"Test failed with unexpected error: {e}") def test_enrichment_with_external_pg(self, mock_stdout): db_adapter = DatabaseTypeAdapter.POSTGRESQL with EnrichmentTestHelpers.sql_test_context(False, db_adapter): - try: - enrichment_with_external_pg() - output = mock_stdout.getvalue().splitlines() - expected = validate_enrichment_with_sql() - self.assertEqual(output, expected) - except Exception as e: - self.fail(f"Test failed with unexpected error: {e}") + try: + enrichment_with_external_pg() + output = mock_stdout.getvalue().splitlines() + expected = validate_enrichment_with_sql() + self.assertEqual(output, expected) + except Exception as e: + self.fail(f"Test failed with unexpected error: {e}") def test_enrichment_with_external_mysql(self, mock_stdout): db_adapter = DatabaseTypeAdapter.MYSQL with EnrichmentTestHelpers.sql_test_context(False, db_adapter): - try: - enrichment_with_external_mysql() - output = mock_stdout.getvalue().splitlines() - expected = validate_enrichment_with_sql() - self.assertEqual(output, expected) - except Exception as e: - self.fail(f"Test failed with unexpected error: {e}") + try: + enrichment_with_external_mysql() + output = mock_stdout.getvalue().splitlines() + expected = validate_enrichment_with_sql() + self.assertEqual(output, expected) + except Exception as e: + self.fail(f"Test failed with unexpected error: {e}") def test_enrichment_with_external_sqlserver(self, mock_stdout): db_adapter = DatabaseTypeAdapter.SQLSERVER with EnrichmentTestHelpers.sql_test_context(False, db_adapter): - try: - enrichment_with_external_sqlserver() - output = mock_stdout.getvalue().splitlines() - expected = validate_enrichment_with_sql() - self.assertEqual(output, expected) - except Exception as e: - self.fail(f"Test failed with unexpected error: {e}") + try: + enrichment_with_external_sqlserver() + output = mock_stdout.getvalue().splitlines() + expected = validate_enrichment_with_sql() + self.assertEqual(output, expected) + except Exception as e: + self.fail(f"Test failed with unexpected error: {e}") @dataclass @@ -166,13 +165,14 @@ class CloudSQLEnrichmentTestDataConstruct: metadata: MetaData db: SQLDBContainerInfo = None + class EnrichmentTestHelpers: @contextmanager def sql_test_context(is_cloudsql: bool, db_adapter: DatabaseTypeAdapter): result: Optional[CloudSQLEnrichmentTestDataConstruct] = None try: result = EnrichmentTestHelpers.pre_sql_enrichment_test( - is_cloudsql, db_adapter) + is_cloudsql, db_adapter) yield finally: if result: @@ -180,8 +180,8 @@ def sql_test_context(is_cloudsql: bool, db_adapter: DatabaseTypeAdapter): @staticmethod def pre_sql_enrichment_test( - is_cloudsql: bool, - db_adapter: DatabaseTypeAdapter) -> CloudSQLEnrichmentTestDataConstruct: + is_cloudsql: bool, + db_adapter: DatabaseTypeAdapter) -> CloudSQLEnrichmentTestDataConstruct: table_id = "products_catalog" columns = [ Column("product_id", Integer, primary_key=True), @@ -251,10 +251,10 @@ def pre_sql_enrichment_test( metadata=metadata) result = CloudSQLEnrichmentTestDataConstruct( - db=db, - client_handler=sql_client_handler, - engine=engine, - metadata=metadata) + db=db, + client_handler=sql_client_handler, + engine=engine, + metadata=metadata) return result @staticmethod diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py index bf812b0931c3..b86c939d37f0 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py @@ -122,7 +122,7 @@ def get_db_url(self) -> str: @dataclass class CloudSQLConnectionConfig(ConnectionConfig): - """Connects to Google Cloud SQL using Cloud SQL Python Connector. + """Connects to Google Cloud SQL using Cloud SQL Python Connector. Args: db_adapter: The database adapter type (PostgreSQL, MySQL, SQL Server). @@ -136,46 +136,42 @@ class CloudSQLConnectionConfig(ConnectionConfig): connect_kwargs: Additional keyword arguments for the client connect method. Enables forward compatibility. """ - db_adapter: DatabaseTypeAdapter - instance_connection_uri: str - user: str = field(default_factory=str) - password: str = field(default_factory=str) - db_id: str = field(default_factory=str) - refresh_strategy: RefreshStrategy = RefreshStrategy.LAZY - connector_kwargs: Dict[str, Any] = field(default_factory=dict) - connect_kwargs: Dict[str, Any] = field(default_factory=dict) - - def __post_init__(self): - if not self.instance_connection_uri: - raise ValueError("Instance connection URI cannot be empty") - - def get_connector_handler(self) -> Callable[[], DBAPIConnection]: - """Returns a function that creates a new database connection. + db_adapter: DatabaseTypeAdapter + instance_connection_uri: str + user: str = field(default_factory=str) + password: str = field(default_factory=str) + db_id: str = field(default_factory=str) + refresh_strategy: RefreshStrategy = RefreshStrategy.LAZY + connector_kwargs: Dict[str, Any] = field(default_factory=dict) + connect_kwargs: Dict[str, Any] = field(default_factory=dict) + + def __post_init__(self): + if not self.instance_connection_uri: + raise ValueError("Instance connection URI cannot be empty") + + def get_connector_handler(self) -> Callable[[], DBAPIConnection]: + """Returns a function that creates a new database connection. The returned connector function creates database connections that should be properly closed by the caller when no longer needed. """ - cloudsql_client = CloudSQLConnector( - refresh_strategy=self.refresh_strategy, - **self.connector_kwargs) + cloudsql_client = CloudSQLConnector( + refresh_strategy=self.refresh_strategy, **self.connector_kwargs) - cloudsql_connector = lambda: cloudsql_client.connect( - instance_connection_string=self.instance_connection_uri, - driver=self.db_adapter.value, - user=self.user, - password=self.password, - db=self.db_id, - **self.connect_kwargs) + cloudsql_connector = lambda: cloudsql_client.connect( + instance_connection_string=self.instance_connection_uri, driver=self. + db_adapter.value, user=self.user, password=self.password, db=self.db_id, + **self.connect_kwargs) - return cloudsql_connector + return cloudsql_connector - def get_db_url(self) -> str: - return self.db_adapter.to_sqlalchemy_dialect() + "://" + def get_db_url(self) -> str: + return self.db_adapter.to_sqlalchemy_dialect() + "://" @dataclass class ExternalSQLDBConnectionConfig(ConnectionConfig): - """Connects to External SQL DBs (PostgreSQL, MySQL, SQL Server) over TCP. + """Connects to External SQL DBs (PostgreSQL, MySQL, SQL Server) over TCP. Args: db_adapter: The database adapter type (PostgreSQL, MySQL, SQL Server). @@ -187,40 +183,40 @@ class ExternalSQLDBConnectionConfig(ConnectionConfig): connect_kwargs: Additional keyword arguments for the client connect method. Enables forward compatibility. """ - db_adapter: DatabaseTypeAdapter - host: str - port: int - user: str = field(default_factory=str) - password: str = field(default_factory=str) - db_id: str = field(default_factory=str) - connect_kwargs: Dict[str, Any] = field(default_factory=dict) + db_adapter: DatabaseTypeAdapter + host: str + port: int + user: str = field(default_factory=str) + password: str = field(default_factory=str) + db_id: str = field(default_factory=str) + connect_kwargs: Dict[str, Any] = field(default_factory=dict) - def __post_init__(self): - if not self.host: - raise ValueError("Database host cannot be empty") + def __post_init__(self): + if not self.host: + raise ValueError("Database host cannot be empty") - def get_connector_handler(self) -> Callable[[], DBAPIConnection]: - """Returns a function that creates a new database connection. + def get_connector_handler(self) -> Callable[[], DBAPIConnection]: + """Returns a function that creates a new database connection. The returned connector function creates database connections that should be properly closed by the caller when no longer needed. """ - if self.db_adapter == DatabaseTypeAdapter.POSTGRESQL: - sql_connector = lambda: pg8000.connect( - host=self.host, port=self.port, database=self.db_id, - user=self.user, password=self.password, **self.connect_kwargs) - elif self.db_adapter == DatabaseTypeAdapter.MYSQL: - sql_connector = lambda: pymysql.connect( - host=self.host, port=self.port, database=self.db_id, - user=self.user, password=self.password, **self.connect_kwargs) - elif self.db_adapter == DatabaseTypeAdapter.SQLSERVER: - sql_connector = lambda: pytds.connect( - dsn=self.host, port=self.port, database=self.db_id, user=self.user, - password=self.password, **self.connect_kwargs) - return sql_connector - - def get_db_url(self) -> str: - return self.db_adapter.to_sqlalchemy_dialect() + "://" + if self.db_adapter == DatabaseTypeAdapter.POSTGRESQL: + sql_connector = lambda: pg8000.connect( + host=self.host, port=self.port, database=self.db_id, user=self.user, + password=self.password, **self.connect_kwargs) + elif self.db_adapter == DatabaseTypeAdapter.MYSQL: + sql_connector = lambda: pymysql.connect( + host=self.host, port=self.port, database=self.db_id, user=self.user, + password=self.password, **self.connect_kwargs) + elif self.db_adapter == DatabaseTypeAdapter.SQLSERVER: + sql_connector = lambda: pytds.connect( + dsn=self.host, port=self.port, database=self.db_id, user=self.user, + password=self.password, **self.connect_kwargs) + return sql_connector + + def get_db_url(self) -> str: + return self.db_adapter.to_sqlalchemy_dialect() + "://" QueryConfig = Union[CustomQueryConfig, @@ -317,30 +313,29 @@ def __init__( def __enter__(self): connector = self._connection_config.get_connector_handler() self._engine = create_engine( - url=self._connection_config.get_db_url(), - creator=connector) + url=self._connection_config.get_db_url(), creator=connector) def _execute_query(self, query: str, is_batch: bool, **params): try: connection = self._engine.connect() transaction = connection.begin() try: - result = connection.execute(text(query), **params) - # Materialize results while transaction is active. - if is_batch: - data = [row._asdict() for row in result] - else: - data = result.first()._asdict() - # Explicitly commit the transaction. - transaction.commit() - return data + result = connection.execute(text(query), **params) + # Materialize results while transaction is active. + if is_batch: + data = [row._asdict() for row in result] + else: + data = result.first()._asdict() + # Explicitly commit the transaction. + transaction.commit() + return data except Exception as e: transaction.rollback() raise RuntimeError(f"Database operation failed: {e}") except Exception as e: raise Exception( - f'Could not execute the query: {query}. Please check if ' - f'the query is properly formatted and the table exists. {e}') + f'Could not execute the query: {query}. Please check if ' + f'the query is properly formatted and the table exists. {e}') finally: if connection: connection.close() diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py index 69d489011bcf..886ae8a54df3 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py @@ -192,12 +192,12 @@ def create_table( # Create contextual connection for schema creation. with engine.connect() as schema_connection: - try: - metadata.create_all(schema_connection) - schema_connection.commit() - except Exception as e: - schema_connection.rollback() - raise RuntimeError(f"Failed to create table schema: {e}") + try: + metadata.create_all(schema_connection) + schema_connection.commit() + except Exception as e: + schema_connection.rollback() + raise RuntimeError(f"Failed to create table schema: {e}") # Now create a separate contextual connection for data insertion. with engine.connect() as connection: @@ -208,6 +208,7 @@ def create_table( connection.rollback() raise Exception(f"Failed to insert table data: {e}") + @pytest.mark.uses_testcontainer class BaseTestSQLEnrichment(unittest.TestCase): _table_data = [ @@ -251,8 +252,7 @@ def setUpClass(cls): connector = cls._connection_config.get_connector_handler() cls._engine = create_engine( - url=cls._connection_config.get_db_url(), - creator=connector) + url=cls._connection_config.get_db_url(), creator=connector) SQLEnrichmentTestHelper.create_table( table_id=cls._table_id, @@ -452,7 +452,7 @@ def test_sql_enrichment_on_non_existent_table(self): column_names=["wrong_column"], ) - with self.assertRaises(Exception) as context: + with self.assertRaises(Exception) as context: with TestPipeline() as p: _ = (p | beam.Create(requests) | Enrichment(handler)) @@ -573,8 +573,7 @@ def setUpClass(cls): # Type hint data from subclasses. cls._db_adapter: DatabaseTypeAdapter - cls._db = SQLEnrichmentTestHelper.start_sql_db_container( - cls._db_adapter) + cls._db = SQLEnrichmentTestHelper.start_sql_db_container(cls._db_adapter) cls._connection_config = ExternalSQLDBConnectionConfig( db_adapter=cls._db_adapter, host=cls._db.host, diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_test.py index 6cca33b9a3c0..00223db707b7 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_test.py @@ -57,7 +57,7 @@ def test_invalid_cloudsql_db_connection_params(self): password='', db_id='') self.assertIn( - "Instance connection URI cannot be empty", str(context.exception)) + "Instance connection URI cannot be empty", str(context.exception)) @parameterized.expand([ # Empty TableFieldsQueryConfig. From b2202c4dcd5a8db729654f6d6a660c37c7cb74a8 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sat, 28 Jun 2025 22:03:04 +0000 Subject: [PATCH 30/83] .github: trigger post commits python --- .github/trigger_files/beam_PostCommit_Python.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/trigger_files/beam_PostCommit_Python.json b/.github/trigger_files/beam_PostCommit_Python.json index 1eb60f6e4959..72105e8e3646 100644 --- a/.github/trigger_files/beam_PostCommit_Python.json +++ b/.github/trigger_files/beam_PostCommit_Python.json @@ -1,5 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run.", - "modification": 3 + "modification": 10 } From b06eff78fcf6072e7c0abde262814e3597a32fce Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sat, 28 Jun 2025 23:42:01 +0000 Subject: [PATCH 31/83] sdks/python+website: update docs and fix linting issues --- .../transforms/elementwise/enrichment.py | 4 +- .../transforms/elementwise/enrichment_test.py | 49 ++++++++-- .../python/elementwise/enrichment-cloudsql.md | 97 +++++++++++++++++-- .../python/elementwise/enrichment.md | 2 +- .../section-menu/en/documentation.html | 1 + 5 files changed, 133 insertions(+), 20 deletions(-) diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py index 4df3d2c5d9a6..e8589fcc8280 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py @@ -12,7 +12,7 @@ # 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 +# See the License for the specific langutage governing permissions and # limitations under the License. # @@ -324,6 +324,6 @@ def enrichment_with_external_sqlserver(): _ = ( p | "Create" >> beam.Create(data) - | "Enrich W/ Unmanaged SQLServer" >> Enrichment(cloudsql_handler) + | "Enrich W/ Unmanaged SQL Server" >> Enrichment(cloudsql_handler) | "Print" >> beam.Map(print)) # [END enrichment_with_external_sqlserver] diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py index 4138b3e0af2c..19ff5efda5b4 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py @@ -18,6 +18,7 @@ # pytype: skip-file # pylint: disable=line-too-long +from collections.abc import Callable from contextlib import contextmanager import os from typing import Optional @@ -27,6 +28,7 @@ import mock import pytest +from sqlalchemy.engine import Connection as DBAPIConnection # pylint: disable=unused-import try: @@ -41,7 +43,7 @@ enrichment_with_external_mysql, enrichment_with_external_sqlserver) from apache_beam.transforms.enrichment_handlers.cloudsql import ( - DatabaseTypeAdapter, SQLClientConnectionHandler) + DatabaseTypeAdapter) from apache_beam.transforms.enrichment_handlers.cloudsql_it_test import ( SQLEnrichmentTestHelper, SQLDBContainerInfo, @@ -80,12 +82,39 @@ def validate_enrichment_with_vertex_ai_legacy(): return expected -def validate_enrichment_with_sql(): - expected = '''[START enrichment_with_sql] +def validate_enrichment_with_google_cloudsql_pg(): + expected = '''[START enrichment_with_google_cloudsql_pg] Row(product_id=1, name='A', quantity=2, region_id=3) Row(product_id=2, name='B', quantity=3, region_id=1) Row(product_id=3, name='C', quantity=10, region_id=4) - [END enrichment_with_sql]'''.splitlines()[1:-1] + [END enrichment_with_google_cloudsql_pg]'''.splitlines()[1:-1] + return expected + + +def validate_enrichment_with_external_pg(): + expected = '''[START enrichment_with_external_pg] +Row(product_id=1, name='A', quantity=2, region_id=3) +Row(product_id=2, name='B', quantity=3, region_id=1) +Row(product_id=3, name='C', quantity=10, region_id=4) + [END enrichment_with_external_pg]'''.splitlines()[1:-1] + return expected + + +def validate_enrichment_with_external_mysql(): + expected = '''[START enrichment_with_external_mysql] +Row(product_id=1, name='A', quantity=2, region_id=3) +Row(product_id=2, name='B', quantity=3, region_id=1) +Row(product_id=3, name='C', quantity=10, region_id=4) + [END enrichment_with_external_mysql]'''.splitlines()[1:-1] + return expected + + +def validate_enrichment_with_external_sqlserver(): + expected = '''[START enrichment_with_external_sqlserver] +Row(product_id=1, name='A', quantity=2, region_id=3) +Row(product_id=2, name='B', quantity=3, region_id=1) +Row(product_id=3, name='C', quantity=10, region_id=4) + [END enrichment_with_external_sqlserver]'''.splitlines()[1:-1] return expected @@ -119,7 +148,7 @@ def test_enrichment_with_google_cloudsql_pg(self, mock_stdout): try: enrichment_with_google_cloudsql_pg() output = mock_stdout.getvalue().splitlines() - expected = validate_enrichment_with_sql() + expected = validate_enrichment_with_google_cloudsql_pg() self.assertEqual(output, expected) except Exception as e: self.fail(f"Test failed with unexpected error: {e}") @@ -130,7 +159,7 @@ def test_enrichment_with_external_pg(self, mock_stdout): try: enrichment_with_external_pg() output = mock_stdout.getvalue().splitlines() - expected = validate_enrichment_with_sql() + expected = validate_enrichment_with_external_pg() self.assertEqual(output, expected) except Exception as e: self.fail(f"Test failed with unexpected error: {e}") @@ -141,7 +170,7 @@ def test_enrichment_with_external_mysql(self, mock_stdout): try: enrichment_with_external_mysql() output = mock_stdout.getvalue().splitlines() - expected = validate_enrichment_with_sql() + expected = validate_enrichment_with_external_mysql() self.assertEqual(output, expected) except Exception as e: self.fail(f"Test failed with unexpected error: {e}") @@ -152,7 +181,7 @@ def test_enrichment_with_external_sqlserver(self, mock_stdout): try: enrichment_with_external_sqlserver() output = mock_stdout.getvalue().splitlines() - expected = validate_enrichment_with_sql() + expected = validate_enrichment_with_external_sqlserver() self.assertEqual(output, expected) except Exception as e: self.fail(f"Test failed with unexpected error: {e}") @@ -160,7 +189,7 @@ def test_enrichment_with_external_sqlserver(self, mock_stdout): @dataclass class CloudSQLEnrichmentTestDataConstruct: - client_handler: SQLClientConnectionHandler + client_handler: Callable[[], DBAPIConnection] engine: Engine metadata: MetaData db: SQLDBContainerInfo = None @@ -182,7 +211,7 @@ def sql_test_context(is_cloudsql: bool, db_adapter: DatabaseTypeAdapter): def pre_sql_enrichment_test( is_cloudsql: bool, db_adapter: DatabaseTypeAdapter) -> CloudSQLEnrichmentTestDataConstruct: - table_id = "products_catalog" + table_id = "products" columns = [ Column("product_id", Integer, primary_key=True), Column("name", VARCHAR(255), nullable=False), diff --git a/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment-cloudsql.md b/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment-cloudsql.md index a8c5de1c2694..8dfc5a174186 100644 --- a/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment-cloudsql.md +++ b/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment-cloudsql.md @@ -29,12 +29,95 @@ limitations under the License. -In Apache Beam and later versions, the enrichment transform includes -a built-in enrichment handler for -[CloudSQL](https://cloud.google.com/sql/docs). -The following example demonstrates how to create a pipeline that use the enrichment transform with the [`CloudSQLEnrichmentHandler`](https://beam.apache.org/releases/pydoc/current/apache_beam.transforms.enrichment_handlers.cloudsql.html#apache_beam.transforms.enrichment_handlers.cloudsql.CloudSQLEnrichmentHandler) handler. +Starting with Apache Beam 2.67.0, the enrichment transform includes +built-in enrichment handler support for the +[Google CloudSQL](https://cloud.google.com/sql/docs). This handler allows your +Beam pipeline to enrich data using SQL databases, with built-in support for: -The data in the CloudSQL PostgreSQL products table instance follows this format: +- Managed PostgreSQL, MySQL, and Microsoft SQL Server instances on CloudSQL +- Unmanaged SQL database instances not hosted on CloudSQL (e.g., self-hosted or + on-premises databases) + +The following example demonstrates how to create a pipeline that use the +enrichment transform with the +[`CloudSQLEnrichmentHandler`](https://beam.apache.org/releases/pydoc/current/apache_beam.transforms.enrichment_handlers.cloudsql.html#apache_beam.transforms.enrichment_handlers.cloudsql.CloudSQLEnrichmentHandler) handler. + +## Example 1: Enrichment with Google CloudSQL (Managed PostgreSQL) + +The data in the CloudSQL PostgreSQL table `products` follows this format: + +{{< table >}} +| product_id | name | quantity | region_id | +|:----------:|:----:|:--------:|:---------:| +| 1 | A | 2 | 3 | +| 2 | B | 3 | 1 | +| 3 | C | 10 | 4 | +{{< /table >}} + + +{{< highlight language="py" >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py" enrichment_with_google_cloudsql_pg >}} +{{}} + +{{< paragraph class="notebook-skip" >}} +Output: +{{< /paragraph >}} +{{< highlight class="notebook-skip" >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py" enrichment_with_google_cloudsql_pg >}} +{{< /highlight >}} + +## Example 2: Enrichment with Unmanaged PostgreSQL + +The data in the Unmanaged PostgreSQL table `products` follows this format: + +{{< table >}} +| product_id | name | quantity | region_id | +|:----------:|:----:|:--------:|:---------:| +| 1 | A | 2 | 3 | +| 2 | B | 3 | 1 | +| 3 | C | 10 | 4 | +{{< /table >}} + + +{{< highlight language="py" >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py" enrichment_with_external_pg >}} +{{}} + +{{< paragraph class="notebook-skip" >}} +Output: +{{< /paragraph >}} +{{< highlight class="notebook-skip" >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py" enrichment_with_external_pg >}} +{{< /highlight >}} + +## Example 3: Enrichment with Unmanaged MySQL + +The data in the Unmanaged MySQL table `products` follows this format: + +{{< table >}} +| product_id | name | quantity | region_id | +|:----------:|:----:|:--------:|:---------:| +| 1 | A | 2 | 3 | +| 2 | B | 3 | 1 | +| 3 | C | 10 | 4 | +{{< /table >}} + + +{{< highlight language="py" >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py" enrichment_with_external_mysql >}} +{{}} + +{{< paragraph class="notebook-skip" >}} +Output: +{{< /paragraph >}} +{{< highlight class="notebook-skip" >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py" enrichment_with_external_mysql >}} +{{< /highlight >}} + +## Example 4: Enrichment with Unmanaged Microsoft SQL Server + +The data in the Unmanaged Microsoft SQL Server table `products` follows this +format: {{< table >}} | product_id | name | quantity | region_id | @@ -46,14 +129,14 @@ The data in the CloudSQL PostgreSQL products table instance follows this format: {{< highlight language="py" >}} -{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py" enrichment_with_cloudsql >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py" enrichment_with_external_sqlserver >}} {{}} {{< paragraph class="notebook-skip" >}} Output: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py" enrichment_with_cloudsql >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py" enrichment_with_external_sqlserver >}} {{< /highlight >}} ## Related transforms diff --git a/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment.md b/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment.md index 0993963ec057..bdf3f9938d0b 100644 --- a/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment.md +++ b/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment.md @@ -42,7 +42,7 @@ The following examples demonstrate how to create a pipeline that use the enrichm | Service | Example | |:-----------------------------------|:-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------| | Cloud Bigtable | [Enrichment with Bigtable](/documentation/transforms/python/elementwise/enrichment-bigtable/#example) | -| Cloud SQL | [Enrichment with CloudSQL](/documentation/transforms/python/elementwise/enrichment-cloudsql/#example) | +| Cloud SQL (PostgreSQL, MySQL, SQLServer) | [Enrichment with CloudSQL](/documentation/transforms/python/elementwise/enrichment-cloudsql/#example) | | Vertex AI Feature Store | [Enrichment with Vertex AI Feature Store](/documentation/transforms/python/elementwise/enrichment-vertexai/#example-1-enrichment-with-vertex-ai-feature-store) | | Vertex AI Feature Store (Legacy) | [Enrichment with Legacy Vertex AI Feature Store](/documentation/transforms/python/elementwise/enrichment-vertexai/#example-2-enrichment-with-vertex-ai-feature-store-legacy) | {{< /table >}} diff --git a/website/www/site/layouts/partials/section-menu/en/documentation.html b/website/www/site/layouts/partials/section-menu/en/documentation.html index 6b37450786f9..9fb842228e30 100755 --- a/website/www/site/layouts/partials/section-menu/en/documentation.html +++ b/website/www/site/layouts/partials/section-menu/en/documentation.html @@ -297,6 +297,7 @@ From 2abba914f71128f4f7df80065976b29a8f4927e6 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sat, 28 Jun 2025 23:50:28 +0000 Subject: [PATCH 32/83] CHANGES.md: update release notes --- CHANGES.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/CHANGES.md b/CHANGES.md index 5cde77cfebf8..3b9b2d44daa8 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -75,6 +75,10 @@ * X feature added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). * Add pip-based install support for JupyterLab Sidepanel extension ([#35397](https://github.com/apache/beam/issues/#35397)). +* Google CloudSQL enrichment handler added (Python) ([#34398](https://github.com/apache/beam/pull/34398)). + Beam now supports data enrichment capabilities using SQL databases, with built-in support for: + - Managed PostgreSQL, MySQL, and Microsoft SQL Server instances on CloudSQL + - Unmanaged SQL database instances not hosted on CloudSQL (e.g., self-hosted or on-premises databases) ## Breaking Changes From 3cd98e413f8bd5e3b18b56c62c6e7d76c4dc9f14 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sat, 28 Jun 2025 23:51:22 +0000 Subject: [PATCH 33/83] .github: trigger postcommit python CI --- .github/trigger_files/beam_PostCommit_Python.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/trigger_files/beam_PostCommit_Python.json b/.github/trigger_files/beam_PostCommit_Python.json index 72105e8e3646..62905b12a707 100644 --- a/.github/trigger_files/beam_PostCommit_Python.json +++ b/.github/trigger_files/beam_PostCommit_Python.json @@ -1,5 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run.", - "modification": 10 + "modification": 13 } From ce0da2f5a7b96cb92a015fcfb6c24dca786d5e33 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 29 Jun 2025 00:46:53 +0000 Subject: [PATCH 34/83] sdks/python: fix linting issues --- .../snippets/transforms/elementwise/enrichment_test.py | 4 ++-- .../transforms/enrichment_handlers/cloudsql_it_test.py | 1 - 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py index 19ff5efda5b4..82fca9b2fce5 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py @@ -267,10 +267,10 @@ def pre_sql_enrichment_test( password=db.password, db_id=db.id) - sql_client_handler = connection_config.get_connector_handler() + conenctor = connection_config.get_connector_handler() engine = create_engine( url=connection_config.get_db_url(), - creator=sql_client_handler.connector) + creator=conenctor) SQLEnrichmentTestHelper.create_table( table_id=table_id, diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py index 886ae8a54df3..8b76d3e87fab 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py @@ -20,7 +20,6 @@ import unittest from dataclasses import dataclass from typing import Optional -from typing import Tuple from unittest.mock import MagicMock import pytest From 9b546abdfb18772ca35eac0771ab15fce644d046 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 29 Jun 2025 00:47:23 +0000 Subject: [PATCH 35/83] .github: trigger postcommit python CI --- .github/trigger_files/beam_PostCommit_Python.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/trigger_files/beam_PostCommit_Python.json b/.github/trigger_files/beam_PostCommit_Python.json index 62905b12a707..a52ee0807ad2 100644 --- a/.github/trigger_files/beam_PostCommit_Python.json +++ b/.github/trigger_files/beam_PostCommit_Python.json @@ -1,5 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run.", - "modification": 13 + "modification": 17 } From 5931d65cbbc25f9eda2fef64f74cda2a57dff49c Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 29 Jun 2025 00:59:04 +0000 Subject: [PATCH 36/83] sdks/python: fix linting issues --- .../snippets/transforms/elementwise/enrichment_test.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py index 82fca9b2fce5..415a4f9f50d3 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py @@ -269,8 +269,7 @@ def pre_sql_enrichment_test( conenctor = connection_config.get_connector_handler() engine = create_engine( - url=connection_config.get_db_url(), - creator=conenctor) + url=connection_config.get_db_url(), creator=conenctor) SQLEnrichmentTestHelper.create_table( table_id=table_id, From e57cba6fc7a3a5eb895c7f9a9677d95406802441 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 29 Jun 2025 00:59:28 +0000 Subject: [PATCH 37/83] .github: trigger postcommit python CI --- .github/trigger_files/beam_PostCommit_Python.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/trigger_files/beam_PostCommit_Python.json b/.github/trigger_files/beam_PostCommit_Python.json index a52ee0807ad2..9ff4de3c230a 100644 --- a/.github/trigger_files/beam_PostCommit_Python.json +++ b/.github/trigger_files/beam_PostCommit_Python.json @@ -1,5 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run.", - "modification": 17 + "modification": 21 } From 7e13c17ded3895ca980e4b7d1f9fbc2fbcfaa280 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 29 Jun 2025 01:19:33 +0000 Subject: [PATCH 38/83] sdks/python: fix `examples` CI for cloudSQL enrichment --- .../snippets/transforms/elementwise/enrichment_test.py | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py index 415a4f9f50d3..3846cbfae424 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py @@ -279,10 +279,7 @@ def pre_sql_enrichment_test( metadata=metadata) result = CloudSQLEnrichmentTestDataConstruct( - db=db, - client_handler=sql_client_handler, - engine=engine, - metadata=metadata) + db=db, client_handler=conenctor, engine=engine, metadata=metadata) return result @staticmethod From 37eeec2f5e5a2142e795db984c77e36df754984a Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 29 Jun 2025 01:20:59 +0000 Subject: [PATCH 39/83] .github: trigger postcommit python CI --- .github/trigger_files/beam_PostCommit_Python.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/trigger_files/beam_PostCommit_Python.json b/.github/trigger_files/beam_PostCommit_Python.json index 9ff4de3c230a..f6e2913a2113 100644 --- a/.github/trigger_files/beam_PostCommit_Python.json +++ b/.github/trigger_files/beam_PostCommit_Python.json @@ -1,5 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run.", - "modification": 21 + "modification": 25 } From 73b55ff7d0eb7a1a14b524ee61f07f92f6fb36c8 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 29 Jun 2025 01:40:55 +0000 Subject: [PATCH 40/83] .github/workflows: include `ALLOYDB_PASSWORD` env in precommit python examples --- .github/workflows/beam_PreCommit_Python_Examples.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/beam_PreCommit_Python_Examples.yml b/.github/workflows/beam_PreCommit_Python_Examples.yml index c76d140eadeb..3b7681faa892 100644 --- a/.github/workflows/beam_PreCommit_Python_Examples.yml +++ b/.github/workflows/beam_PreCommit_Python_Examples.yml @@ -53,6 +53,7 @@ env: DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + ALLOYDB_PASSWORD: ${{ secrets.ALLOYDB_PASSWORD }} jobs: beam_PreCommit_Python_Examples: From a6775a79f47780dac3542776d1ee3b412f3c1510 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 29 Jun 2025 01:44:21 +0000 Subject: [PATCH 41/83] sdks/python: fix linting issues for `cloudsql` --- .../transforms/enrichment_handlers/cloudsql.py | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py index b86c939d37f0..12647a839b54 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py @@ -14,24 +14,27 @@ # See the License for the specific language governing permissions and # limitations under the License. # -from abc import abstractmethod, ABC +from abc import ABC +from abc import abstractmethod from collections.abc import Callable from collections.abc import Mapping -from dataclasses import dataclass, field +from dataclasses import dataclass +from dataclasses import field from enum import Enum -from typing import Any, Dict +from typing import Any +from typing import Dict from typing import List from typing import Optional from typing import Union -import pymysql import pg8000 +import pymysql import pytds +from google.cloud.sql.connector import Connector as CloudSQLConnector +from google.cloud.sql.connector.enums import RefreshStrategy from sqlalchemy import create_engine from sqlalchemy import text from sqlalchemy.engine import Connection as DBAPIConnection -from google.cloud.sql.connector.enums import RefreshStrategy -from google.cloud.sql.connector import Connector as CloudSQLConnector import apache_beam as beam from apache_beam.transforms.enrichment import EnrichmentSourceHandler From 82602ed9e6b9cf6767d7632aacde455673c1de5f Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 29 Jun 2025 02:08:30 +0000 Subject: [PATCH 42/83] sdks/python: fix linting issues for `enrichment_test.py` Signed-off-by: Mohamed Awnallah --- .../snippets/transforms/elementwise/enrichment_test.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py index 3846cbfae424..1b3fdd36423c 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py @@ -18,13 +18,13 @@ # pytype: skip-file # pylint: disable=line-too-long -from collections.abc import Callable -from contextlib import contextmanager import os -from typing import Optional import unittest -from io import StringIO +from collections.abc import Callable +from contextlib import contextmanager from dataclasses import dataclass +from io import StringIO +from typing import Optional import mock import pytest From c9062098b627342cd1b2f5118f8ffbf6ac8f299b Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 29 Jun 2025 02:09:41 +0000 Subject: [PATCH 43/83] .github: trigger postcommit --- .github/trigger_files/beam_PostCommit_Python.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/trigger_files/beam_PostCommit_Python.json b/.github/trigger_files/beam_PostCommit_Python.json index f6e2913a2113..00e0c3c25433 100644 --- a/.github/trigger_files/beam_PostCommit_Python.json +++ b/.github/trigger_files/beam_PostCommit_Python.json @@ -1,5 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run.", - "modification": 25 + "modification": 27 } From 7a92a99392e9383e17620162f0261a604b6504a4 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 29 Jun 2025 02:57:55 +0000 Subject: [PATCH 44/83] .github+website+sdks/python: remove non-functional docs --- .../beam_PreCommit_Python_Examples.yml | 1 - .../transforms/elementwise/enrichment.py | 213 +--------------- .../transforms/elementwise/enrichment_test.py | 231 +----------------- .../python/elementwise/enrichment-cloudsql.md | 146 ----------- .../python/elementwise/enrichment.md | 1 - .../section-menu/en/documentation.html | 1 - 6 files changed, 6 insertions(+), 587 deletions(-) delete mode 100644 website/www/site/content/en/documentation/transforms/python/elementwise/enrichment-cloudsql.md diff --git a/.github/workflows/beam_PreCommit_Python_Examples.yml b/.github/workflows/beam_PreCommit_Python_Examples.yml index 3b7681faa892..c76d140eadeb 100644 --- a/.github/workflows/beam_PreCommit_Python_Examples.yml +++ b/.github/workflows/beam_PreCommit_Python_Examples.yml @@ -53,7 +53,6 @@ env: DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} - ALLOYDB_PASSWORD: ${{ secrets.ALLOYDB_PASSWORD }} jobs: beam_PreCommit_Python_Examples: diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py index e8589fcc8280..acee633b6f67 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py @@ -12,7 +12,7 @@ # 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 langutage governing permissions and +# See the License for the specific language governing permissions and # limitations under the License. # @@ -116,214 +116,3 @@ def enrichment_with_vertex_ai_legacy(): | "Enrich W/ Vertex AI" >> Enrichment(vertex_ai_handler) | "Print" >> beam.Map(print)) # [END enrichment_with_vertex_ai_legacy] - - -def enrichment_with_google_cloudsql_pg(): - # [START enrichment_with_google_cloudsql_pg] - import apache_beam as beam - from apache_beam.transforms.enrichment import Enrichment - from apache_beam.transforms.enrichment_handlers.cloudsql import ( - CloudSQLEnrichmentHandler, - DatabaseTypeAdapter, - TableFieldsQueryConfig, - CloudSQLConnectionConfig) - import os - - database_adapter = DatabaseTypeAdapter.POSTGRESQL - database_uri = os.environ.get("GOOGLE_CLOUD_SQL_DB_URI") - database_user = int(os.environ.get("GOOGLE_CLOUD_SQL_DB_USER")) - database_password = os.environ.get("GOOGLE_CLOUD_SQL_DB_PASSWORD") - database_id = os.environ.get("GOOGLE_CLOUD_SQL_DB_ID") - table_id = os.environ.get("GOOGLE_CLOUD_SQL_DB_TABLE_ID") - where_clause_template = "product_id = {}" - where_clause_fields = ["product_id"] - - data = [ - beam.Row(product_id=1, name='A'), - beam.Row(product_id=2, name='B'), - beam.Row(product_id=3, name='C'), - ] - - connection_config = CloudSQLConnectionConfig( - db_adapter=database_adapter, - instance_connection_uri=database_uri, - user=database_user, - password=database_password, - db_id=database_id) - - query_config = TableFieldsQueryConfig( - table_id=table_id, - where_clause_template=where_clause_template, - where_clause_fields=where_clause_fields) - - cloudsql_handler = CloudSQLEnrichmentHandler( - connection_config=connection_config, - table_id=table_id, - query_config=query_config) - with beam.Pipeline() as p: - _ = ( - p - | "Create" >> beam.Create(data) - | - "Enrich W/ Google CloudSQL PostgreSQL" >> Enrichment(cloudsql_handler) - | "Print" >> beam.Map(print)) - # [END enrichment_with_google_cloudsql_pg] - - -def enrichment_with_external_pg(): - # [START enrichment_with_external_pg] - import apache_beam as beam - from apache_beam.transforms.enrichment import Enrichment - from apache_beam.transforms.enrichment_handlers.cloudsql import ( - CloudSQLEnrichmentHandler, - DatabaseTypeAdapter, - TableFieldsQueryConfig, - ExternalSQLDBConnectionConfig) - import os - - database_adapter = DatabaseTypeAdapter.POSTGRESQL - database_host = os.environ.get("EXTERNAL_SQL_DB_HOST") - database_port = int(os.environ.get("EXTERNAL_SQL_DB_PORT")) - database_user = os.environ.get("EXTERNAL_SQL_DB_USER") - database_password = os.environ.get("EXTERNAL_SQL_DB_PASSWORD") - database_id = os.environ.get("EXTERNAL_SQL_DB_ID") - table_id = os.environ.get("EXTERNAL_SQL_DB_TABLE_ID") - where_clause_template = "product_id = {}" - where_clause_fields = ["product_id"] - - data = [ - beam.Row(product_id=1, name='A'), - beam.Row(product_id=2, name='B'), - beam.Row(product_id=3, name='C'), - ] - - connection_config = ExternalSQLDBConnectionConfig( - db_adapter=database_adapter, - host=database_host, - port=database_port, - user=database_user, - password=database_password, - db_id=database_id) - - query_config = TableFieldsQueryConfig( - table_id=table_id, - where_clause_template=where_clause_template, - where_clause_fields=where_clause_fields) - - cloudsql_handler = CloudSQLEnrichmentHandler( - connection_config=connection_config, - table_id=table_id, - query_config=query_config) - with beam.Pipeline() as p: - _ = ( - p - | "Create" >> beam.Create(data) - | "Enrich W/ Unmanaged PostgreSQL" >> Enrichment(cloudsql_handler) - | "Print" >> beam.Map(print)) - # [END enrichment_with_external_pg] - - -def enrichment_with_external_mysql(): - # [START enrichment_with_external_mysql] - import apache_beam as beam - from apache_beam.transforms.enrichment import Enrichment - from apache_beam.transforms.enrichment_handlers.cloudsql import ( - CloudSQLEnrichmentHandler, - DatabaseTypeAdapter, - TableFieldsQueryConfig, - ExternalSQLDBConnectionConfig) - import os - - database_adapter = DatabaseTypeAdapter.MYSQL - database_host = os.environ.get("EXTERNAL_SQL_DB_HOST") - database_port = int(os.environ.get("EXTERNAL_SQL_DB_PORT")) - database_user = os.environ.get("EXTERNAL_SQL_DB_USER") - database_password = os.environ.get("EXTERNAL_SQL_DB_PASSWORD") - database_id = os.environ.get("EXTERNAL_SQL_DB_ID") - table_id = os.environ.get("EXTERNAL_SQL_DB_TABLE_ID") - where_clause_template = "product_id = {}" - where_clause_fields = ["product_id"] - - data = [ - beam.Row(product_id=1, name='A'), - beam.Row(product_id=2, name='B'), - beam.Row(product_id=3, name='C'), - ] - - connection_config = ExternalSQLDBConnectionConfig( - db_adapter=database_adapter, - host=database_host, - port=database_port, - user=database_user, - password=database_password, - db_id=database_id) - - query_config = TableFieldsQueryConfig( - table_id=table_id, - where_clause_template=where_clause_template, - where_clause_fields=where_clause_fields) - - cloudsql_handler = CloudSQLEnrichmentHandler( - connection_config=connection_config, - table_id=table_id, - query_config=query_config) - with beam.Pipeline() as p: - _ = ( - p - | "Create" >> beam.Create(data) - | "Enrich W/ Unmanaged MySQL" >> Enrichment(cloudsql_handler) - | "Print" >> beam.Map(print)) - # [END enrichment_with_external_mysql] - - -def enrichment_with_external_sqlserver(): - # [START enrichment_with_external_sqlserver] - import apache_beam as beam - from apache_beam.transforms.enrichment import Enrichment - from apache_beam.transforms.enrichment_handlers.cloudsql import ( - CloudSQLEnrichmentHandler, - DatabaseTypeAdapter, - TableFieldsQueryConfig, - ExternalSQLDBConnectionConfig) - import os - - database_adapter = DatabaseTypeAdapter.SQLSERVER - database_host = os.environ.get("EXTERNAL_SQL_DB_HOST") - database_port = int(os.environ.get("EXTERNAL_SQL_DB_PORT")) - database_user = os.environ.get("EXTERNAL_SQL_DB_USER") - database_password = os.environ.get("EXTERNAL_SQL_DB_PASSWORD") - database_id = os.environ.get("EXTERNAL_SQL_DB_ID") - table_id = os.environ.get("EXTERNAL_SQL_DB_TABLE_ID") - where_clause_template = "product_id = {}" - where_clause_fields = ["product_id"] - - data = [ - beam.Row(product_id=1, name='A'), - beam.Row(product_id=2, name='B'), - beam.Row(product_id=3, name='C'), - ] - - connection_config = ExternalSQLDBConnectionConfig( - db_adapter=database_adapter, - host=database_host, - port=database_port, - user=database_user, - password=database_password, - db_id=database_id) - - query_config = TableFieldsQueryConfig( - table_id=table_id, - where_clause_template=where_clause_template, - where_clause_fields=where_clause_fields) - - cloudsql_handler = CloudSQLEnrichmentHandler( - connection_config=connection_config, - table_id=table_id, - query_config=query_config) - with beam.Pipeline() as p: - _ = ( - p - | "Create" >> beam.Create(data) - | "Enrich W/ Unmanaged SQL Server" >> Enrichment(cloudsql_handler) - | "Print" >> beam.Map(print)) - # [END enrichment_with_external_sqlserver] diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py index 1b3fdd36423c..8a7cdfbe9263 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py @@ -18,41 +18,19 @@ # pytype: skip-file # pylint: disable=line-too-long -import os import unittest -from collections.abc import Callable -from contextlib import contextmanager -from dataclasses import dataclass from io import StringIO -from typing import Optional import mock -import pytest -from sqlalchemy.engine import Connection as DBAPIConnection # pylint: disable=unused-import try: - from sqlalchemy import ( - Column, Integer, VARCHAR, Engine, MetaData, create_engine) - from apache_beam.examples.snippets.transforms.elementwise.enrichment import ( - enrichment_with_bigtable, enrichment_with_vertex_ai_legacy) - from apache_beam.examples.snippets.transforms.elementwise.enrichment import ( - enrichment_with_vertex_ai, - enrichment_with_google_cloudsql_pg, - enrichment_with_external_pg, - enrichment_with_external_mysql, - enrichment_with_external_sqlserver) - from apache_beam.transforms.enrichment_handlers.cloudsql import ( - DatabaseTypeAdapter) - from apache_beam.transforms.enrichment_handlers.cloudsql_it_test import ( - SQLEnrichmentTestHelper, - SQLDBContainerInfo, - ConnectionConfig, - CloudSQLConnectionConfig, - ExternalSQLDBConnectionConfig) + from apache_beam.examples.snippets.transforms.elementwise.enrichment import enrichment_with_bigtable, \ + enrichment_with_vertex_ai_legacy + from apache_beam.examples.snippets.transforms.elementwise.enrichment import enrichment_with_vertex_ai from apache_beam.io.requestresponse import RequestResponseIO -except ImportError as e: - raise unittest.SkipTest(f'RequestResponseIO dependencies not installed: {e}') +except ImportError: + raise unittest.SkipTest('RequestResponseIO dependencies are not installed') def validate_enrichment_with_bigtable(): @@ -82,44 +60,7 @@ def validate_enrichment_with_vertex_ai_legacy(): return expected -def validate_enrichment_with_google_cloudsql_pg(): - expected = '''[START enrichment_with_google_cloudsql_pg] -Row(product_id=1, name='A', quantity=2, region_id=3) -Row(product_id=2, name='B', quantity=3, region_id=1) -Row(product_id=3, name='C', quantity=10, region_id=4) - [END enrichment_with_google_cloudsql_pg]'''.splitlines()[1:-1] - return expected - - -def validate_enrichment_with_external_pg(): - expected = '''[START enrichment_with_external_pg] -Row(product_id=1, name='A', quantity=2, region_id=3) -Row(product_id=2, name='B', quantity=3, region_id=1) -Row(product_id=3, name='C', quantity=10, region_id=4) - [END enrichment_with_external_pg]'''.splitlines()[1:-1] - return expected - - -def validate_enrichment_with_external_mysql(): - expected = '''[START enrichment_with_external_mysql] -Row(product_id=1, name='A', quantity=2, region_id=3) -Row(product_id=2, name='B', quantity=3, region_id=1) -Row(product_id=3, name='C', quantity=10, region_id=4) - [END enrichment_with_external_mysql]'''.splitlines()[1:-1] - return expected - - -def validate_enrichment_with_external_sqlserver(): - expected = '''[START enrichment_with_external_sqlserver] -Row(product_id=1, name='A', quantity=2, region_id=3) -Row(product_id=2, name='B', quantity=3, region_id=1) -Row(product_id=3, name='C', quantity=10, region_id=4) - [END enrichment_with_external_sqlserver]'''.splitlines()[1:-1] - return expected - - @mock.patch('sys.stdout', new_callable=StringIO) -@pytest.mark.uses_testcontainer class EnrichmentTest(unittest.TestCase): def test_enrichment_with_bigtable(self, mock_stdout): enrichment_with_bigtable() @@ -142,168 +83,6 @@ def test_enrichment_with_vertex_ai_legacy(self, mock_stdout): self.maxDiff = None self.assertEqual(output, expected) - def test_enrichment_with_google_cloudsql_pg(self, mock_stdout): - db_adapter = DatabaseTypeAdapter.POSTGRESQL - with EnrichmentTestHelpers.sql_test_context(True, db_adapter): - try: - enrichment_with_google_cloudsql_pg() - output = mock_stdout.getvalue().splitlines() - expected = validate_enrichment_with_google_cloudsql_pg() - self.assertEqual(output, expected) - except Exception as e: - self.fail(f"Test failed with unexpected error: {e}") - - def test_enrichment_with_external_pg(self, mock_stdout): - db_adapter = DatabaseTypeAdapter.POSTGRESQL - with EnrichmentTestHelpers.sql_test_context(False, db_adapter): - try: - enrichment_with_external_pg() - output = mock_stdout.getvalue().splitlines() - expected = validate_enrichment_with_external_pg() - self.assertEqual(output, expected) - except Exception as e: - self.fail(f"Test failed with unexpected error: {e}") - - def test_enrichment_with_external_mysql(self, mock_stdout): - db_adapter = DatabaseTypeAdapter.MYSQL - with EnrichmentTestHelpers.sql_test_context(False, db_adapter): - try: - enrichment_with_external_mysql() - output = mock_stdout.getvalue().splitlines() - expected = validate_enrichment_with_external_mysql() - self.assertEqual(output, expected) - except Exception as e: - self.fail(f"Test failed with unexpected error: {e}") - - def test_enrichment_with_external_sqlserver(self, mock_stdout): - db_adapter = DatabaseTypeAdapter.SQLSERVER - with EnrichmentTestHelpers.sql_test_context(False, db_adapter): - try: - enrichment_with_external_sqlserver() - output = mock_stdout.getvalue().splitlines() - expected = validate_enrichment_with_external_sqlserver() - self.assertEqual(output, expected) - except Exception as e: - self.fail(f"Test failed with unexpected error: {e}") - - -@dataclass -class CloudSQLEnrichmentTestDataConstruct: - client_handler: Callable[[], DBAPIConnection] - engine: Engine - metadata: MetaData - db: SQLDBContainerInfo = None - - -class EnrichmentTestHelpers: - @contextmanager - def sql_test_context(is_cloudsql: bool, db_adapter: DatabaseTypeAdapter): - result: Optional[CloudSQLEnrichmentTestDataConstruct] = None - try: - result = EnrichmentTestHelpers.pre_sql_enrichment_test( - is_cloudsql, db_adapter) - yield - finally: - if result: - EnrichmentTestHelpers.post_sql_enrichment_test(result) - - @staticmethod - def pre_sql_enrichment_test( - is_cloudsql: bool, - db_adapter: DatabaseTypeAdapter) -> CloudSQLEnrichmentTestDataConstruct: - table_id = "products" - columns = [ - Column("product_id", Integer, primary_key=True), - Column("name", VARCHAR(255), nullable=False), - Column("quantity", Integer, nullable=False), - Column("region_id", Integer, nullable=False), - ] - table_data = [ - { - "product_id": 1, "name": "A", 'quantity': 2, 'region_id': 3 - }, - { - "product_id": 2, "name": "B", 'quantity': 3, 'region_id': 1 - }, - { - "product_id": 3, "name": "C", 'quantity': 10, 'region_id': 4 - }, - ] - metadata = MetaData() - - connection_config: ConnectionConfig - if is_cloudsql: - gcp_project_id = "apache-beam-testing" - region = "us-central1" - instance_name = "beam-integration-tests" - instance_connection_uri = f"{gcp_project_id}:{region}:{instance_name}" - db_id = "postgres" - user = "postgres" - password = os.getenv("ALLOYDB_PASSWORD") - os.environ['GOOGLE_CLOUD_SQL_DB_URI'] = instance_connection_uri - os.environ['GOOGLE_CLOUD_SQL_DB_ID'] = db_id - os.environ['GOOGLE_CLOUD_SQL_DB_USER'] = user - os.environ['GOOGLE_CLOUD_SQL_DB_PASSWORD'] = password - os.environ['GOOGLE_CLOUD_SQL_DB_TABLE_ID'] = table_id - connection_config = CloudSQLConnectionConfig( - db_adapter=db_adapter, - instance_connection_uri=instance_connection_uri, - user=user, - password=password, - db_id=db_id) - else: - db = SQLEnrichmentTestHelper.start_sql_db_container(db_adapter) - os.environ['EXTERNAL_SQL_DB_HOST'] = db.host - os.environ['EXTERNAL_SQL_DB_PORT'] = str(db.port) - os.environ['EXTERNAL_SQL_DB_ID'] = db.id - os.environ['EXTERNAL_SQL_DB_USER'] = db.user - os.environ['EXTERNAL_SQL_DB_PASSWORD'] = db.password - os.environ['EXTERNAL_SQL_DB_TABLE_ID'] = table_id - connection_config = ExternalSQLDBConnectionConfig( - db_adapter=db_adapter, - host=db.host, - port=db.port, - user=db.user, - password=db.password, - db_id=db.id) - - conenctor = connection_config.get_connector_handler() - engine = create_engine( - url=connection_config.get_db_url(), creator=conenctor) - - SQLEnrichmentTestHelper.create_table( - table_id=table_id, - engine=engine, - columns=columns, - table_data=table_data, - metadata=metadata) - - result = CloudSQLEnrichmentTestDataConstruct( - db=db, client_handler=conenctor, engine=engine, metadata=metadata) - return result - - @staticmethod - def post_sql_enrichment_test(res: CloudSQLEnrichmentTestDataConstruct): - # Clean up the data inserted previously. - res.metadata.drop_all(res.engine) - res.engine.dispose(close=True) - - # Check if the test used a container-based external SQL database. - if res.db: - SQLEnrichmentTestHelper.stop_sql_db_container(res.db) - os.environ.pop('EXTERNAL_SQL_DB_HOST', None) - os.environ.pop('EXTERNAL_SQL_DB_PORT', None) - os.environ.pop('EXTERNAL_SQL_DB_ID', None) - os.environ.pop('EXTERNAL_SQL_DB_USER', None) - os.environ.pop('EXTERNAL_SQL_DB_PASSWORD', None) - os.environ.pop('EXTERNAL_SQL_DB_TABLE_ID', None) - else: - os.environ.pop('GOOGLE_CLOUD_SQL_DB_URI', None) - os.environ.pop('GOOGLE_CLOUD_SQL_DB_ID', None) - os.environ.pop('GOOGLE_CLOUD_SQL_DB_USER', None) - os.environ.pop('GOOGLE_CLOUD_SQL_DB_PASSWORD', None) - os.environ.pop('GOOGLE_CLOUD_SQL_DB_TABLE_ID', None) - if __name__ == '__main__': unittest.main() diff --git a/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment-cloudsql.md b/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment-cloudsql.md deleted file mode 100644 index 8dfc5a174186..000000000000 --- a/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment-cloudsql.md +++ /dev/null @@ -1,146 +0,0 @@ ---- -title: "Enrichment with CloudSQL" ---- - - -# Use CloudSQL to enrich data - -{{< localstorage language language-py >}} - - - - - -
- - {{< button-pydoc path="apache_beam.transforms.enrichment_handlers.cloudsql" class="CloudSQLEnrichmentHandler" >}} - -
- -Starting with Apache Beam 2.67.0, the enrichment transform includes -built-in enrichment handler support for the -[Google CloudSQL](https://cloud.google.com/sql/docs). This handler allows your -Beam pipeline to enrich data using SQL databases, with built-in support for: - -- Managed PostgreSQL, MySQL, and Microsoft SQL Server instances on CloudSQL -- Unmanaged SQL database instances not hosted on CloudSQL (e.g., self-hosted or - on-premises databases) - -The following example demonstrates how to create a pipeline that use the -enrichment transform with the -[`CloudSQLEnrichmentHandler`](https://beam.apache.org/releases/pydoc/current/apache_beam.transforms.enrichment_handlers.cloudsql.html#apache_beam.transforms.enrichment_handlers.cloudsql.CloudSQLEnrichmentHandler) handler. - -## Example 1: Enrichment with Google CloudSQL (Managed PostgreSQL) - -The data in the CloudSQL PostgreSQL table `products` follows this format: - -{{< table >}} -| product_id | name | quantity | region_id | -|:----------:|:----:|:--------:|:---------:| -| 1 | A | 2 | 3 | -| 2 | B | 3 | 1 | -| 3 | C | 10 | 4 | -{{< /table >}} - - -{{< highlight language="py" >}} -{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py" enrichment_with_google_cloudsql_pg >}} -{{}} - -{{< paragraph class="notebook-skip" >}} -Output: -{{< /paragraph >}} -{{< highlight class="notebook-skip" >}} -{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py" enrichment_with_google_cloudsql_pg >}} -{{< /highlight >}} - -## Example 2: Enrichment with Unmanaged PostgreSQL - -The data in the Unmanaged PostgreSQL table `products` follows this format: - -{{< table >}} -| product_id | name | quantity | region_id | -|:----------:|:----:|:--------:|:---------:| -| 1 | A | 2 | 3 | -| 2 | B | 3 | 1 | -| 3 | C | 10 | 4 | -{{< /table >}} - - -{{< highlight language="py" >}} -{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py" enrichment_with_external_pg >}} -{{}} - -{{< paragraph class="notebook-skip" >}} -Output: -{{< /paragraph >}} -{{< highlight class="notebook-skip" >}} -{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py" enrichment_with_external_pg >}} -{{< /highlight >}} - -## Example 3: Enrichment with Unmanaged MySQL - -The data in the Unmanaged MySQL table `products` follows this format: - -{{< table >}} -| product_id | name | quantity | region_id | -|:----------:|:----:|:--------:|:---------:| -| 1 | A | 2 | 3 | -| 2 | B | 3 | 1 | -| 3 | C | 10 | 4 | -{{< /table >}} - - -{{< highlight language="py" >}} -{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py" enrichment_with_external_mysql >}} -{{}} - -{{< paragraph class="notebook-skip" >}} -Output: -{{< /paragraph >}} -{{< highlight class="notebook-skip" >}} -{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py" enrichment_with_external_mysql >}} -{{< /highlight >}} - -## Example 4: Enrichment with Unmanaged Microsoft SQL Server - -The data in the Unmanaged Microsoft SQL Server table `products` follows this -format: - -{{< table >}} -| product_id | name | quantity | region_id | -|:----------:|:----:|:--------:|:---------:| -| 1 | A | 2 | 3 | -| 2 | B | 3 | 1 | -| 3 | C | 10 | 4 | -{{< /table >}} - - -{{< highlight language="py" >}} -{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py" enrichment_with_external_sqlserver >}} -{{}} - -{{< paragraph class="notebook-skip" >}} -Output: -{{< /paragraph >}} -{{< highlight class="notebook-skip" >}} -{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py" enrichment_with_external_sqlserver >}} -{{< /highlight >}} - -## Related transforms - -Not applicable. - -{{< button-pydoc path="apache_beam.transforms.enrichment_handlers.cloudsql" class="CloudSQLEnrichmentHandler" >}} \ No newline at end of file diff --git a/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment.md b/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment.md index bdf3f9938d0b..6c05b6b515a4 100644 --- a/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment.md +++ b/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment.md @@ -42,7 +42,6 @@ The following examples demonstrate how to create a pipeline that use the enrichm | Service | Example | |:-----------------------------------|:-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------| | Cloud Bigtable | [Enrichment with Bigtable](/documentation/transforms/python/elementwise/enrichment-bigtable/#example) | -| Cloud SQL (PostgreSQL, MySQL, SQLServer) | [Enrichment with CloudSQL](/documentation/transforms/python/elementwise/enrichment-cloudsql/#example) | | Vertex AI Feature Store | [Enrichment with Vertex AI Feature Store](/documentation/transforms/python/elementwise/enrichment-vertexai/#example-1-enrichment-with-vertex-ai-feature-store) | | Vertex AI Feature Store (Legacy) | [Enrichment with Legacy Vertex AI Feature Store](/documentation/transforms/python/elementwise/enrichment-vertexai/#example-2-enrichment-with-vertex-ai-feature-store-legacy) | {{< /table >}} diff --git a/website/www/site/layouts/partials/section-menu/en/documentation.html b/website/www/site/layouts/partials/section-menu/en/documentation.html index 9fb842228e30..6b37450786f9 100755 --- a/website/www/site/layouts/partials/section-menu/en/documentation.html +++ b/website/www/site/layouts/partials/section-menu/en/documentation.html @@ -297,7 +297,6 @@ From da35a23b27fc7dbc39a44bbae14ee0069a83081e Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 29 Jun 2025 02:58:53 +0000 Subject: [PATCH 45/83] .github: trigger postcommit python CI --- .github/trigger_files/beam_PostCommit_Python.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/trigger_files/beam_PostCommit_Python.json b/.github/trigger_files/beam_PostCommit_Python.json index 00e0c3c25433..42a6e88b8a29 100644 --- a/.github/trigger_files/beam_PostCommit_Python.json +++ b/.github/trigger_files/beam_PostCommit_Python.json @@ -1,5 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run.", - "modification": 27 + "modification": 31 } From 44aac808b2ccbcf2f5d1d3aba47cb43527631dd0 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Wed, 2 Jul 2025 18:24:56 +0000 Subject: [PATCH 46/83] sdks/python: generate unique table for itest --- .../enrichment_handlers/cloudsql_it_test.py | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py index 8b76d3e87fab..89643ca68310 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py @@ -21,6 +21,7 @@ from dataclasses import dataclass from typing import Optional from unittest.mock import MagicMock +import uuid import pytest @@ -266,7 +267,7 @@ def setUpClass(cls): def get_columns(cls): """Returns fresh column objects each time it's called.""" return [ - Column("id", Integer, primary_key=True), + Column("id", Integer, nullable=False), Column("name", VARCHAR(255), nullable=False), Column("quantity", Integer, nullable=False), Column("distribution_center_id", Integer, nullable=False), @@ -592,21 +593,24 @@ def tearDownClass(cls): @pytest.mark.uses_testcontainer class TestExternalPostgresEnrichment(BaseExternalSQLDBEnrichment): _db_adapter = DatabaseTypeAdapter.POSTGRESQL - _table_id = "product_details_external_pg_enrichment" + _unique_suffix = str(uuid.uuid4())[:8] + _table_id = f"product_details_external_pg_enrichment_{_unique_suffix}" _metadata = MetaData() @pytest.mark.uses_testcontainer class TestExternalMySQLEnrichment(BaseExternalSQLDBEnrichment): _db_adapter = DatabaseTypeAdapter.MYSQL - _table_id = "product_details_external_mysql_enrichment" + _unique_suffix = str(uuid.uuid4())[:8] + _table_id = f"product_details_external_mysql_enrichment_{_unique_suffix}" _metadata = MetaData() @pytest.mark.uses_testcontainer class TestExternalSQLServerEnrichment(BaseExternalSQLDBEnrichment): _db_adapter = DatabaseTypeAdapter.SQLSERVER - _table_id = "product_details_external_mssql_enrichment" + _unique_suffix = str(uuid.uuid4())[:8] + _table_id = f"product_details_external_mssql_enrichment_{_unique_suffix}" _metadata = MetaData() From e0affe248ce8a659d8e61a1e200f1aea34295f7c Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Wed, 2 Jul 2025 18:25:57 +0000 Subject: [PATCH 47/83] .github: enable access to `ALLOYDB_PASSWORD` in precommit py transforms --- .github/workflows/beam_PreCommit_Python_Transforms.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/beam_PreCommit_Python_Transforms.yml b/.github/workflows/beam_PreCommit_Python_Transforms.yml index 1a16e9b61756..8753777057c6 100644 --- a/.github/workflows/beam_PreCommit_Python_Transforms.yml +++ b/.github/workflows/beam_PreCommit_Python_Transforms.yml @@ -53,6 +53,7 @@ env: DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + ALLOYDB_PASSWORD: ${{ secrets.ALLOYDB_PASSWORD }} jobs: beam_PreCommit_Python_Transforms: From 38263c2fa16efbb604a53ac0b38c032d4c3e01b9 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Wed, 2 Jul 2025 18:26:43 +0000 Subject: [PATCH 48/83] .github: trigger post commit python --- .github/trigger_files/beam_PostCommit_Python.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/trigger_files/beam_PostCommit_Python.json b/.github/trigger_files/beam_PostCommit_Python.json index 42a6e88b8a29..4db5f0675328 100644 --- a/.github/trigger_files/beam_PostCommit_Python.json +++ b/.github/trigger_files/beam_PostCommit_Python.json @@ -1,5 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run.", - "modification": 31 + "modification": 37 } From f1e78b8cbf610bafeab0b46f2d7d9c75ac8d6ae3 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Wed, 2 Jul 2025 18:34:49 +0000 Subject: [PATCH 49/83] sdks/python: update py39 image requirements --- .../py39/base_image_requirements.txt | 19 ------------------- 1 file changed, 19 deletions(-) diff --git a/sdks/python/container/py39/base_image_requirements.txt b/sdks/python/container/py39/base_image_requirements.txt index 8657c5c78490..db7961186ba3 100644 --- a/sdks/python/container/py39/base_image_requirements.txt +++ b/sdks/python/container/py39/base_image_requirements.txt @@ -78,11 +78,7 @@ google-cloud-storage==2.19.0 google-cloud-videointelligence==2.16.2 google-cloud-vision==3.10.2 google-crc32c==1.7.1 -<<<<<<< HEAD -google-genai==1.22.0 -======= google-genai==1.23.0 ->>>>>>> upstream/master google-resumable-media==2.7.2 googleapis-common-protos==1.70.0 greenlet==3.2.3 @@ -96,11 +92,7 @@ hdfs==2.7.3 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -<<<<<<< HEAD -hypothesis==6.135.16 -======= hypothesis==6.135.20 ->>>>>>> upstream/master idna==3.10 importlib_metadata==8.7.0 iniconfig==2.1.0 @@ -113,10 +105,6 @@ joblib==1.5.1 jsonpickle==3.4.2 jsonschema==4.24.0 jsonschema-specifications==2025.4.1 -<<<<<<< HEAD -kafka-python==2.2.14 -======= ->>>>>>> upstream/master keyring==25.6.0 keyrings.google-artifactregistry-auth==1.1.2 MarkupSafe==3.0.2 @@ -124,11 +112,7 @@ milvus-lite==2.5.1 mmh3==5.1.0 mock==5.2.0 more-itertools==10.7.0 -<<<<<<< HEAD -multidict==6.6.0 -======= multidict==6.6.3 ->>>>>>> upstream/master mysql-connector-python==9.3.0 nltk==3.9.1 numpy==2.0.2 @@ -160,10 +144,7 @@ pydantic_core==2.33.2 pydot==1.4.2 PyHamcrest==2.1.0 PyJWT==2.9.0 -<<<<<<< HEAD -======= pymilvus==2.5.11 ->>>>>>> upstream/master pymongo==4.13.2 PyMySQL==1.1.1 pyparsing==3.2.3 From fc2e073c94bfdcc7a772bb0880cbd4bbfd8cbf28 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Wed, 2 Jul 2025 18:35:47 +0000 Subject: [PATCH 50/83] .github: trigger postcommit python --- .github/trigger_files/beam_PostCommit_Python.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/trigger_files/beam_PostCommit_Python.json b/.github/trigger_files/beam_PostCommit_Python.json index 4db5f0675328..5bf57b1aa5bc 100644 --- a/.github/trigger_files/beam_PostCommit_Python.json +++ b/.github/trigger_files/beam_PostCommit_Python.json @@ -1,5 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run.", - "modification": 37 + "modification": 41 } From a95d9846538d157f4451e622e0bbb9897daddd67 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Wed, 2 Jul 2025 21:54:26 +0000 Subject: [PATCH 51/83] sdks/python: update image requirements --- .../container/py310/base_image_requirements.txt | 14 +++++++------- .../container/py310/ml_image_requirements.txt | 16 ++++++++-------- .../container/py311/base_image_requirements.txt | 14 +++++++------- .../container/py311/ml_image_requirements.txt | 16 ++++++++-------- .../container/py312/base_image_requirements.txt | 17 ++++++++++------- .../container/py312/ml_image_requirements.txt | 16 ++++++++-------- .../container/py39/base_image_requirements.txt | 14 +++++++------- .../container/py39/ml_image_requirements.txt | 16 ++++++++-------- 8 files changed, 63 insertions(+), 60 deletions(-) diff --git a/sdks/python/container/py310/base_image_requirements.txt b/sdks/python/container/py310/base_image_requirements.txt index 81834540267c..c9ad8779e695 100644 --- a/sdks/python/container/py310/base_image_requirements.txt +++ b/sdks/python/container/py310/base_image_requirements.txt @@ -41,7 +41,7 @@ charset-normalizer==3.4.2 click==8.2.1 cloud-sql-python-connector==1.18.2 crcmod==1.7 -cryptography==45.0.4 +cryptography==45.0.5 Cython==3.1.2 dill==0.3.1.1 dnspython==2.7.0 @@ -60,7 +60,7 @@ google-api-python-client==2.174.0 google-apitools==0.5.31 google-auth==2.40.3 google-auth-httplib2==0.2.0 -google-cloud-aiplatform==1.100.0 +google-cloud-aiplatform==1.101.0 google-cloud-bigquery==3.34.0 google-cloud-bigquery-storage==2.32.0 google-cloud-bigtable==2.31.0 @@ -78,7 +78,7 @@ google-cloud-storage==2.19.0 google-cloud-videointelligence==2.16.2 google-cloud-vision==3.10.2 google-crc32c==1.7.1 -google-genai==1.23.0 +google-genai==1.24.0 google-resumable-media==2.7.2 googleapis-common-protos==1.70.0 greenlet==3.2.3 @@ -92,7 +92,7 @@ hdfs==2.7.3 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -hypothesis==6.135.19 +hypothesis==6.135.22 idna==3.10 importlib_metadata==8.7.0 iniconfig==2.1.0 @@ -144,14 +144,14 @@ pydantic_core==2.33.2 pydot==1.4.2 PyHamcrest==2.1.0 PyJWT==2.9.0 -pymilvus==2.5.11 +pymilvus==2.5.12 pymongo==4.13.2 PyMySQL==1.1.1 pyparsing==3.2.3 pyproject_hooks==1.2.0 pytest==7.4.4 pytest-timeout==2.4.0 -pytest-xdist==3.7.0 +pytest-xdist==3.8.0 python-dateutil==2.9.0.post0 python-dotenv==1.1.1 python-tds==1.16.1 @@ -162,7 +162,7 @@ referencing==0.36.2 regex==2024.11.6 requests==2.32.4 requests-mock==1.12.1 -rpds-py==0.25.1 +rpds-py==0.26.0 rsa==4.9.1 scikit-learn==1.7.0 scipy==1.15.3 diff --git a/sdks/python/container/py310/ml_image_requirements.txt b/sdks/python/container/py310/ml_image_requirements.txt index 2b70da331f53..3cc673f6105b 100644 --- a/sdks/python/container/py310/ml_image_requirements.txt +++ b/sdks/python/container/py310/ml_image_requirements.txt @@ -43,7 +43,7 @@ charset-normalizer==3.4.2 click==8.2.1 cloud-sql-python-connector==1.18.2 crcmod==1.7 -cryptography==45.0.4 +cryptography==45.0.5 Cython==3.1.2 dill==0.3.1.1 dnspython==2.7.0 @@ -66,7 +66,7 @@ google-api-python-client==2.174.0 google-apitools==0.5.31 google-auth==2.40.3 google-auth-httplib2==0.2.0 -google-cloud-aiplatform==1.100.0 +google-cloud-aiplatform==1.101.0 google-cloud-bigquery==3.34.0 google-cloud-bigquery-storage==2.32.0 google-cloud-bigtable==2.31.0 @@ -84,7 +84,7 @@ google-cloud-storage==2.19.0 google-cloud-videointelligence==2.16.2 google-cloud-vision==3.10.2 google-crc32c==1.7.1 -google-genai==1.23.0 +google-genai==1.24.0 google-pasta==0.2.0 google-resumable-media==2.7.2 googleapis-common-protos==1.70.0 @@ -101,8 +101,8 @@ hf-xet==1.1.5 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -huggingface-hub==0.33.1 -hypothesis==6.135.19 +huggingface-hub==0.33.2 +hypothesis==6.135.22 idna==3.10 importlib_metadata==8.7.0 iniconfig==2.1.0 @@ -180,14 +180,14 @@ pydot==1.4.2 Pygments==2.19.2 PyHamcrest==2.1.0 PyJWT==2.9.0 -pymilvus==2.5.11 +pymilvus==2.5.12 pymongo==4.13.2 PyMySQL==1.1.1 pyparsing==3.2.3 pyproject_hooks==1.2.0 pytest==7.4.4 pytest-timeout==2.4.0 -pytest-xdist==3.7.0 +pytest-xdist==3.8.0 python-dateutil==2.9.0.post0 python-dotenv==1.1.1 python-tds==1.16.1 @@ -199,7 +199,7 @@ regex==2024.11.6 requests==2.32.4 requests-mock==1.12.1 rich==14.0.0 -rpds-py==0.25.1 +rpds-py==0.26.0 rsa==4.9.1 safetensors==0.5.3 scikit-learn==1.7.0 diff --git a/sdks/python/container/py311/base_image_requirements.txt b/sdks/python/container/py311/base_image_requirements.txt index 2f81ea5e79d1..21304b02ff09 100644 --- a/sdks/python/container/py311/base_image_requirements.txt +++ b/sdks/python/container/py311/base_image_requirements.txt @@ -40,7 +40,7 @@ charset-normalizer==3.4.2 click==8.2.1 cloud-sql-python-connector==1.18.2 crcmod==1.7 -cryptography==45.0.4 +cryptography==45.0.5 Cython==3.1.2 dill==0.3.1.1 dnspython==2.7.0 @@ -58,7 +58,7 @@ google-api-python-client==2.174.0 google-apitools==0.5.31 google-auth==2.40.3 google-auth-httplib2==0.2.0 -google-cloud-aiplatform==1.100.0 +google-cloud-aiplatform==1.101.0 google-cloud-bigquery==3.34.0 google-cloud-bigquery-storage==2.32.0 google-cloud-bigtable==2.31.0 @@ -76,7 +76,7 @@ google-cloud-storage==2.19.0 google-cloud-videointelligence==2.16.2 google-cloud-vision==3.10.2 google-crc32c==1.7.1 -google-genai==1.23.0 +google-genai==1.24.0 google-resumable-media==2.7.2 googleapis-common-protos==1.70.0 greenlet==3.2.3 @@ -90,7 +90,7 @@ hdfs==2.7.3 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -hypothesis==6.135.19 +hypothesis==6.135.22 idna==3.10 importlib_metadata==8.7.0 iniconfig==2.1.0 @@ -142,14 +142,14 @@ pydantic_core==2.33.2 pydot==1.4.2 PyHamcrest==2.1.0 PyJWT==2.9.0 -pymilvus==2.5.11 +pymilvus==2.5.12 pymongo==4.13.2 PyMySQL==1.1.1 pyparsing==3.2.3 pyproject_hooks==1.2.0 pytest==7.4.4 pytest-timeout==2.4.0 -pytest-xdist==3.7.0 +pytest-xdist==3.8.0 python-dateutil==2.9.0.post0 python-dotenv==1.1.1 python-tds==1.16.1 @@ -160,7 +160,7 @@ referencing==0.36.2 regex==2024.11.6 requests==2.32.4 requests-mock==1.12.1 -rpds-py==0.25.1 +rpds-py==0.26.0 rsa==4.9.1 scikit-learn==1.7.0 scipy==1.16.0 diff --git a/sdks/python/container/py311/ml_image_requirements.txt b/sdks/python/container/py311/ml_image_requirements.txt index a5d8add176d2..c7cf86f82f0f 100644 --- a/sdks/python/container/py311/ml_image_requirements.txt +++ b/sdks/python/container/py311/ml_image_requirements.txt @@ -42,7 +42,7 @@ charset-normalizer==3.4.2 click==8.2.1 cloud-sql-python-connector==1.18.2 crcmod==1.7 -cryptography==45.0.4 +cryptography==45.0.5 Cython==3.1.2 dill==0.3.1.1 dnspython==2.7.0 @@ -64,7 +64,7 @@ google-api-python-client==2.174.0 google-apitools==0.5.31 google-auth==2.40.3 google-auth-httplib2==0.2.0 -google-cloud-aiplatform==1.100.0 +google-cloud-aiplatform==1.101.0 google-cloud-bigquery==3.34.0 google-cloud-bigquery-storage==2.32.0 google-cloud-bigtable==2.31.0 @@ -82,7 +82,7 @@ google-cloud-storage==2.19.0 google-cloud-videointelligence==2.16.2 google-cloud-vision==3.10.2 google-crc32c==1.7.1 -google-genai==1.23.0 +google-genai==1.24.0 google-pasta==0.2.0 google-resumable-media==2.7.2 googleapis-common-protos==1.70.0 @@ -99,8 +99,8 @@ hf-xet==1.1.5 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -huggingface-hub==0.33.1 -hypothesis==6.135.19 +huggingface-hub==0.33.2 +hypothesis==6.135.22 idna==3.10 importlib_metadata==8.7.0 iniconfig==2.1.0 @@ -178,14 +178,14 @@ pydot==1.4.2 Pygments==2.19.2 PyHamcrest==2.1.0 PyJWT==2.9.0 -pymilvus==2.5.11 +pymilvus==2.5.12 pymongo==4.13.2 PyMySQL==1.1.1 pyparsing==3.2.3 pyproject_hooks==1.2.0 pytest==7.4.4 pytest-timeout==2.4.0 -pytest-xdist==3.7.0 +pytest-xdist==3.8.0 python-dateutil==2.9.0.post0 python-dotenv==1.1.1 python-tds==1.16.1 @@ -197,7 +197,7 @@ regex==2024.11.6 requests==2.32.4 requests-mock==1.12.1 rich==14.0.0 -rpds-py==0.25.1 +rpds-py==0.26.0 rsa==4.9.1 safetensors==0.5.3 scikit-learn==1.7.0 diff --git a/sdks/python/container/py312/base_image_requirements.txt b/sdks/python/container/py312/base_image_requirements.txt index f010cbde9e64..d7a14ee98532 100644 --- a/sdks/python/container/py312/base_image_requirements.txt +++ b/sdks/python/container/py312/base_image_requirements.txt @@ -39,7 +39,7 @@ charset-normalizer==3.4.2 click==8.2.1 cloud-sql-python-connector==1.18.2 crcmod==1.7 -cryptography==45.0.4 +cryptography==45.0.5 Cython==3.1.2 dill==0.3.1.1 dnspython==2.7.0 @@ -57,7 +57,7 @@ google-api-python-client==2.174.0 google-apitools==0.5.31 google-auth==2.40.3 google-auth-httplib2==0.2.0 -google-cloud-aiplatform==1.100.0 +google-cloud-aiplatform==1.101.0 google-cloud-bigquery==3.34.0 google-cloud-bigquery-storage==2.32.0 google-cloud-bigtable==2.31.0 @@ -75,7 +75,7 @@ google-cloud-storage==2.19.0 google-cloud-videointelligence==2.16.2 google-cloud-vision==3.10.2 google-crc32c==1.7.1 -google-genai==1.23.0 +google-genai==1.24.0 google-resumable-media==2.7.2 googleapis-common-protos==1.70.0 greenlet==3.2.3 @@ -89,7 +89,7 @@ hdfs==2.7.3 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -hypothesis==6.135.19 +hypothesis==6.135.22 idna==3.10 importlib_metadata==8.7.0 iniconfig==2.1.0 @@ -141,11 +141,14 @@ pydantic_core==2.33.2 pydot==1.4.2 PyHamcrest==2.1.0 PyJWT==2.9.0 -pymilvus==2.5.11 +pymilvus==2.5.12 +pymongo==4.13.2 +PyMySQL==1.1.1 pyparsing==3.2.3 +pyproject_hooks==1.2.0 pytest==7.4.4 pytest-timeout==2.4.0 -pytest-xdist==3.7.0 +pytest-xdist==3.8.0 python-dateutil==2.9.0.post0 python-dotenv==1.1.1 python-tds==1.16.1 @@ -156,7 +159,7 @@ referencing==0.36.2 regex==2024.11.6 requests==2.32.4 requests-mock==1.12.1 -rpds-py==0.25.1 +rpds-py==0.26.0 rsa==4.9.1 scikit-learn==1.7.0 scipy==1.16.0 diff --git a/sdks/python/container/py312/ml_image_requirements.txt b/sdks/python/container/py312/ml_image_requirements.txt index e6e9a2930d26..e0e84d4fc4d2 100644 --- a/sdks/python/container/py312/ml_image_requirements.txt +++ b/sdks/python/container/py312/ml_image_requirements.txt @@ -41,7 +41,7 @@ charset-normalizer==3.4.2 click==8.2.1 cloud-sql-python-connector==1.18.2 crcmod==1.7 -cryptography==45.0.4 +cryptography==45.0.5 Cython==3.1.2 dill==0.3.1.1 dnspython==2.7.0 @@ -63,7 +63,7 @@ google-api-python-client==2.174.0 google-apitools==0.5.31 google-auth==2.40.3 google-auth-httplib2==0.2.0 -google-cloud-aiplatform==1.100.0 +google-cloud-aiplatform==1.101.0 google-cloud-bigquery==3.34.0 google-cloud-bigquery-storage==2.32.0 google-cloud-bigtable==2.31.0 @@ -81,7 +81,7 @@ google-cloud-storage==2.19.0 google-cloud-videointelligence==2.16.2 google-cloud-vision==3.10.2 google-crc32c==1.7.1 -google-genai==1.23.0 +google-genai==1.24.0 google-pasta==0.2.0 google-resumable-media==2.7.2 googleapis-common-protos==1.70.0 @@ -98,8 +98,8 @@ hf-xet==1.1.5 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -huggingface-hub==0.33.1 -hypothesis==6.135.20 +huggingface-hub==0.33.2 +hypothesis==6.135.22 idna==3.10 importlib_metadata==8.7.0 iniconfig==2.1.0 @@ -177,14 +177,14 @@ pydot==1.4.2 Pygments==2.19.2 PyHamcrest==2.1.0 PyJWT==2.9.0 -pymilvus==2.5.11 +pymilvus==2.5.12 pymongo==4.13.2 PyMySQL==1.1.1 pyparsing==3.2.3 pyproject_hooks==1.2.0 pytest==7.4.4 pytest-timeout==2.4.0 -pytest-xdist==3.7.0 +pytest-xdist==3.8.0 python-dateutil==2.9.0.post0 python-dotenv==1.1.1 python-tds==1.16.1 @@ -196,7 +196,7 @@ regex==2024.11.6 requests==2.32.4 requests-mock==1.12.1 rich==14.0.0 -rpds-py==0.25.1 +rpds-py==0.26.0 rsa==4.9.1 safetensors==0.5.3 scikit-learn==1.7.0 diff --git a/sdks/python/container/py39/base_image_requirements.txt b/sdks/python/container/py39/base_image_requirements.txt index db7961186ba3..6e6b6c3a2f99 100644 --- a/sdks/python/container/py39/base_image_requirements.txt +++ b/sdks/python/container/py39/base_image_requirements.txt @@ -41,7 +41,7 @@ charset-normalizer==3.4.2 click==8.1.8 cloud-sql-python-connector==1.18.2 crcmod==1.7 -cryptography==45.0.4 +cryptography==45.0.5 Cython==3.1.2 dill==0.3.1.1 dnspython==2.7.0 @@ -60,7 +60,7 @@ google-api-python-client==2.174.0 google-apitools==0.5.31 google-auth==2.40.3 google-auth-httplib2==0.2.0 -google-cloud-aiplatform==1.100.0 +google-cloud-aiplatform==1.101.0 google-cloud-bigquery==3.34.0 google-cloud-bigquery-storage==2.32.0 google-cloud-bigtable==2.31.0 @@ -78,7 +78,7 @@ google-cloud-storage==2.19.0 google-cloud-videointelligence==2.16.2 google-cloud-vision==3.10.2 google-crc32c==1.7.1 -google-genai==1.23.0 +google-genai==1.24.0 google-resumable-media==2.7.2 googleapis-common-protos==1.70.0 greenlet==3.2.3 @@ -92,7 +92,7 @@ hdfs==2.7.3 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -hypothesis==6.135.20 +hypothesis==6.135.23 idna==3.10 importlib_metadata==8.7.0 iniconfig==2.1.0 @@ -144,14 +144,14 @@ pydantic_core==2.33.2 pydot==1.4.2 PyHamcrest==2.1.0 PyJWT==2.9.0 -pymilvus==2.5.11 +pymilvus==2.5.12 pymongo==4.13.2 PyMySQL==1.1.1 pyparsing==3.2.3 pyproject_hooks==1.2.0 pytest==7.4.4 pytest-timeout==2.4.0 -pytest-xdist==3.7.0 +pytest-xdist==3.8.0 python-dateutil==2.9.0.post0 python-dotenv==1.1.1 python-tds==1.16.1 @@ -162,7 +162,7 @@ referencing==0.36.2 regex==2024.11.6 requests==2.32.4 requests-mock==1.12.1 -rpds-py==0.25.1 +rpds-py==0.26.0 rsa==4.9.1 scikit-learn==1.6.1 scipy==1.13.1 diff --git a/sdks/python/container/py39/ml_image_requirements.txt b/sdks/python/container/py39/ml_image_requirements.txt index 3dab7e35b6d1..b78385b1511e 100644 --- a/sdks/python/container/py39/ml_image_requirements.txt +++ b/sdks/python/container/py39/ml_image_requirements.txt @@ -43,7 +43,7 @@ charset-normalizer==3.4.2 click==8.1.8 cloud-sql-python-connector==1.18.2 crcmod==1.7 -cryptography==45.0.4 +cryptography==45.0.5 Cython==3.1.2 dill==0.3.1.1 dnspython==2.7.0 @@ -66,7 +66,7 @@ google-api-python-client==2.174.0 google-apitools==0.5.31 google-auth==2.40.3 google-auth-httplib2==0.2.0 -google-cloud-aiplatform==1.100.0 +google-cloud-aiplatform==1.101.0 google-cloud-bigquery==3.34.0 google-cloud-bigquery-storage==2.32.0 google-cloud-bigtable==2.31.0 @@ -84,7 +84,7 @@ google-cloud-storage==2.19.0 google-cloud-videointelligence==2.16.2 google-cloud-vision==3.10.2 google-crc32c==1.7.1 -google-genai==1.23.0 +google-genai==1.24.0 google-pasta==0.2.0 google-resumable-media==2.7.2 googleapis-common-protos==1.70.0 @@ -101,8 +101,8 @@ hf-xet==1.1.5 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -huggingface-hub==0.33.1 -hypothesis==6.135.20 +huggingface-hub==0.33.2 +hypothesis==6.135.23 idna==3.10 importlib_metadata==8.7.0 iniconfig==2.1.0 @@ -180,14 +180,14 @@ pydot==1.4.2 Pygments==2.19.2 PyHamcrest==2.1.0 PyJWT==2.9.0 -pymilvus==2.5.11 +pymilvus==2.5.12 pymongo==4.13.2 PyMySQL==1.1.1 pyparsing==3.2.3 pyproject_hooks==1.2.0 pytest==7.4.4 pytest-timeout==2.4.0 -pytest-xdist==3.7.0 +pytest-xdist==3.8.0 python-dateutil==2.9.0.post0 python-dotenv==1.1.1 python-tds==1.16.1 @@ -199,7 +199,7 @@ regex==2024.11.6 requests==2.32.4 requests-mock==1.12.1 rich==14.0.0 -rpds-py==0.25.1 +rpds-py==0.26.0 rsa==4.9.1 safetensors==0.5.3 scikit-learn==1.6.1 From 4da20c9802be3a425f4033f7070ade63581f0fda Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Wed, 2 Jul 2025 21:55:02 +0000 Subject: [PATCH 52/83] .github: trigger postcommit python --- .github/trigger_files/beam_PostCommit_Python.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/trigger_files/beam_PostCommit_Python.json b/.github/trigger_files/beam_PostCommit_Python.json index 5bf57b1aa5bc..7c5e53a47167 100644 --- a/.github/trigger_files/beam_PostCommit_Python.json +++ b/.github/trigger_files/beam_PostCommit_Python.json @@ -1,5 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run.", - "modification": 41 + "modification": 62 } From 2f38a596031c4995cf92a839d9c8109673f2c49a Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Tue, 8 Jul 2025 14:54:43 +0000 Subject: [PATCH 53/83] .github: trigger postcommit python --- .github/trigger_files/beam_PostCommit_Python.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/trigger_files/beam_PostCommit_Python.json b/.github/trigger_files/beam_PostCommit_Python.json index 7c5e53a47167..bb5a93d43b09 100644 --- a/.github/trigger_files/beam_PostCommit_Python.json +++ b/.github/trigger_files/beam_PostCommit_Python.json @@ -1,5 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run.", - "modification": 62 + "modification": 70 } From 7819821fb1f661dac44564729e262d4d524484f6 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Tue, 8 Jul 2025 15:27:13 +0000 Subject: [PATCH 54/83] sdks/python: fix local unbound variable --- .../transforms/enrichment_handlers/cloudsql_it_test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py index 89643ca68310..fcc3cc4f79cb 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py @@ -90,6 +90,7 @@ def start_sql_db_container( sql_client_retries=3) -> Optional[SQLDBContainerInfo]: info = None for i in range(sql_client_retries): + sql_db_container = None try: if database_type == DatabaseTypeAdapter.POSTGRESQL: user, password, db_id = "test", "test", "test" From 759f30715ab99adf4c8a4d2a2c811c61ca7dafbe Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Tue, 8 Jul 2025 15:28:42 +0000 Subject: [PATCH 55/83] .github: trigger postcommit python --- .github/trigger_files/beam_PostCommit_Python.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/trigger_files/beam_PostCommit_Python.json b/.github/trigger_files/beam_PostCommit_Python.json index bb5a93d43b09..b17b5793a5b9 100644 --- a/.github/trigger_files/beam_PostCommit_Python.json +++ b/.github/trigger_files/beam_PostCommit_Python.json @@ -1,5 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run.", - "modification": 70 + "modification": 77 } From 47d8da57244d5e86b825cd1caf83a43bb01009fa Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Tue, 8 Jul 2025 17:19:28 +0000 Subject: [PATCH 56/83] sdks/python: remove params --- .../enrichment_handlers/cloudsql_it_test.py | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py index fcc3cc4f79cb..160ee431f779 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py @@ -96,7 +96,7 @@ def start_sql_db_container( user, password, db_id = "test", "test", "test" sql_db_container = PostgresContainer( image="postgres:16", - user=user, + username=user, password=password, dbname=db_id, driver=database_type.value) @@ -108,10 +108,10 @@ def start_sql_db_container( user, password, db_id = "test", "test", "test" sql_db_container = MySqlContainer( image="mysql:8.0", - MYSQL_USER=user, - MYSQL_ROOT_PASSWORD=password, - MYSQL_PASSWORD=password, - MYSQL_DATABASE=db_id) + username=user, + root_password=password, + password=password, + dbname=db_id) sql_db_container.start() host = sql_db_container.get_container_host_ip() port = int(sql_db_container.get_exposed_port(3306)) @@ -120,7 +120,7 @@ def start_sql_db_container( user, password, db_id = "SA", "A_Str0ng_Required_Password", "tempdb" sql_db_container = SqlServerContainer( image="mcr.microsoft.com/mssql/server:2022-latest", - user=user, + username=user, password=password, dbname=db_id, dialect=database_type.to_sqlalchemy_dialect()) From 363873f0b719f08c82b27c37ff0d2da1efed4b18 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Tue, 8 Jul 2025 17:21:03 +0000 Subject: [PATCH 57/83] .github: trigger beam post commit python --- .github/trigger_files/beam_PostCommit_Python.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/trigger_files/beam_PostCommit_Python.json b/.github/trigger_files/beam_PostCommit_Python.json index b17b5793a5b9..259f86cfc284 100644 --- a/.github/trigger_files/beam_PostCommit_Python.json +++ b/.github/trigger_files/beam_PostCommit_Python.json @@ -1,5 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run.", - "modification": 77 + "modification": 79 } From 13ee4f35e3ec7103c2b7e45ec7f0e1b18ec8b357 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Wed, 9 Jul 2025 08:06:19 +0000 Subject: [PATCH 58/83] sdks/python: experiment with my google cloud --- .../transforms/enrichment_handlers/cloudsql_it_test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py index 160ee431f779..baace7dfc6aa 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py @@ -550,14 +550,14 @@ class TestCloudSQLPostgresEnrichment(BaseCloudSQLDBEnrichment): # Configuration required for locating the CloudSQL instance. _table_id = "product_details_cloudsql_pg_enrichment" - _gcp_project_id = "apache-beam-testing" + _gcp_project_id = "cultivated-snow-456016-c0" _region = "us-central1" _instance_name = "beam-integration-tests" _instance_connection_uri = f"{_gcp_project_id}:{_region}:{_instance_name}" # Configuration required for authenticating to the CloudSQL instance. _user = "postgres" - _password = os.getenv("ALLOYDB_PASSWORD") + _password = "password" _db_id = "postgres" _metadata = MetaData() From 6f67233c9bbc5f9c77e1ea6a1caa06abbbdd7de4 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Wed, 9 Jul 2025 08:13:37 +0000 Subject: [PATCH 59/83] sdks/python: revert `ALLOYDB_PASSWORD` env var in precommit py transform workflow --- .github/workflows/beam_PreCommit_Python_Transforms.yml | 1 - 1 file changed, 1 deletion(-) diff --git a/.github/workflows/beam_PreCommit_Python_Transforms.yml b/.github/workflows/beam_PreCommit_Python_Transforms.yml index 8753777057c6..1a16e9b61756 100644 --- a/.github/workflows/beam_PreCommit_Python_Transforms.yml +++ b/.github/workflows/beam_PreCommit_Python_Transforms.yml @@ -53,7 +53,6 @@ env: DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} - ALLOYDB_PASSWORD: ${{ secrets.ALLOYDB_PASSWORD }} jobs: beam_PreCommit_Python_Transforms: From 95214e368f60491adbeb57f39ca35aaa4a8a845d Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Wed, 9 Jul 2025 08:13:58 +0000 Subject: [PATCH 60/83] sdks/python: skip cloudsql tests if `ALLOYDB_PASSWORD` not set --- .../transforms/enrichment_handlers/cloudsql_it_test.py | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py index baace7dfc6aa..a209d9b68410 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py @@ -545,19 +545,22 @@ def tearDownClass(cls): cls._db = None +@unittest.skipUnless( + os.environ.get('ALLOYDB_PASSWORD'), + "ALLOYDB_PASSWORD environment var is not provided") class TestCloudSQLPostgresEnrichment(BaseCloudSQLDBEnrichment): _db_adapter = DatabaseTypeAdapter.POSTGRESQL # Configuration required for locating the CloudSQL instance. _table_id = "product_details_cloudsql_pg_enrichment" - _gcp_project_id = "cultivated-snow-456016-c0" + _gcp_project_id = "apache-beam-testing" _region = "us-central1" _instance_name = "beam-integration-tests" _instance_connection_uri = f"{_gcp_project_id}:{_region}:{_instance_name}" # Configuration required for authenticating to the CloudSQL instance. _user = "postgres" - _password = "password" + _password = os.getenv("ALLOYDB_PASSWORD") _db_id = "postgres" _metadata = MetaData() From 948b8ba7eafd1146756206c65ac55ed7d34311e2 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Wed, 9 Jul 2025 08:20:25 +0000 Subject: [PATCH 61/83] .github: trigger postcommit python --- .github/trigger_files/beam_PostCommit_Python.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/trigger_files/beam_PostCommit_Python.json b/.github/trigger_files/beam_PostCommit_Python.json index 259f86cfc284..69084592b364 100644 --- a/.github/trigger_files/beam_PostCommit_Python.json +++ b/.github/trigger_files/beam_PostCommit_Python.json @@ -1,5 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run.", - "modification": 79 + "modification": 93 } From 09e3606cc86f7b1b13859d264a4c9a8f079e6f59 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 13 Jul 2025 10:39:32 +0000 Subject: [PATCH 62/83] sdks/python: address gemini feedback Co-authored-by: gemini-code-assist[bot] <176961590+gemini-code-assist[bot]@users.noreply.github.com> --- .../enrichment_handlers/cloudsql.py | 24 +++++++++---------- .../enrichment_handlers/cloudsql_it_test.py | 17 ++++++------- 2 files changed, 21 insertions(+), 20 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py index 12647a839b54..2110c51ccac0 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py @@ -63,9 +63,8 @@ class TableFieldsQueryConfig: def __post_init__(self): if not self.table_id or not self.where_clause_template: raise ValueError( - "TableFieldsQueryConfig and " + - "TableFunctionQueryConfig must provide table_id " + - "and where_clause_template") + "TableFieldsQueryConfig must provide table_id and " + + "where_clause_template") if not self.where_clause_fields: raise ValueError( @@ -83,9 +82,8 @@ class TableFunctionQueryConfig: def __post_init__(self): if not self.table_id or not self.where_clause_template: raise ValueError( - "TableFieldsQueryConfig and " + - "TableFunctionQueryConfig must provide table_id " + - "and where_clause_template") + "TableFunctionQueryConfig must provide table_id and " + + "where_clause_template") if not self.where_clause_value_fn: raise ValueError( @@ -264,7 +262,7 @@ def __init__( connection_config = CloudSQLConnectionConfig( db_adapter=DatabaseTypeAdapter.POSTGRESQL, instance_connection_uri="apache-beam-testing:us-central1:itests", - user=postgres, + user='postgres', password= os.getenv("CLOUDSQL_PG_PASSWORD")) query_config=TableFieldsQueryConfig('my_table',"id = '{}'",['id']), cloudsql_handler = CloudSQLEnrichmentHandler( @@ -319,6 +317,7 @@ def __enter__(self): url=self._connection_config.get_db_url(), creator=connector) def _execute_query(self, query: str, is_batch: bool, **params): + connection = None try: connection = self._engine.connect() transaction = connection.begin() @@ -328,7 +327,8 @@ def _execute_query(self, query: str, is_batch: bool, **params): if is_batch: data = [row._asdict() for row in result] else: - data = result.first()._asdict() + result_row = result.first() + data = result_row._asdict() if result_row else {} # Explicitly commit the transaction. transaction.commit() return data @@ -337,8 +337,8 @@ def _execute_query(self, query: str, is_batch: bool, **params): raise RuntimeError(f"Database operation failed: {e}") except Exception as e: raise Exception( - f'Could not execute the query: {query}. Please check if ' - f'the query is properly formatted and the table exists. {e}') + f'Could not execute the query. Please check if the query is properly ' + f'formatted and the table exists. {e}') finally: if connection: connection.close() @@ -454,8 +454,8 @@ def get_cache_key(self, request: Union[beam.Row, list[beam.Row]]): req_dict[field] for field in self._query_config.where_clause_fields ] - key = ";".join(["%s"] * len(current_values)) - cache_keys.extend([key % tuple(current_values)]) + key = ';'.join(map(repr, current_values)) + cache_keys.append(key) except KeyError as e: raise KeyError( "Make sure the values passed in `where_clause_fields` are the " diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py index a209d9b68410..d556948d5104 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py @@ -18,10 +18,10 @@ import logging import os import unittest +import uuid from dataclasses import dataclass from typing import Optional from unittest.mock import MagicMock -import uuid import pytest @@ -103,7 +103,6 @@ def start_sql_db_container( sql_db_container.start() host = sql_db_container.get_container_host_ip() port = int(sql_db_container.get_exposed_port(5432)) - elif database_type == DatabaseTypeAdapter.MYSQL: user, password, db_id = "test", "test", "test" sql_db_container = MySqlContainer( @@ -115,7 +114,6 @@ def start_sql_db_container( sql_db_container.start() host = sql_db_container.get_container_host_ip() port = int(sql_db_container.get_exposed_port(3306)) - elif database_type == DatabaseTypeAdapter.SQLSERVER: user, password, db_id = "SA", "A_Str0ng_Required_Password", "tempdb" sql_db_container = SqlServerContainer( @@ -457,7 +455,9 @@ def test_sql_enrichment_on_non_existent_table(self): with TestPipeline() as p: _ = (p | beam.Create(requests) | Enrichment(handler)) - expect_err_msg_contains = "Could not execute the query" + expect_err_msg_contains = ( + "Could not execute the query. Please check if the query is properly " + "formatted and the table exists.") self.assertIn(expect_err_msg_contains, str(context.exception)) @pytest.mark.usefixtures("cache_container") @@ -498,10 +498,12 @@ def test_sql_enrichment_with_redis(self): if not response: raise ValueError("No cache entry found for %s" % key) - # Mock the CloudSQL enrichment handler to avoid actual database calls. - # This simulates a cache hit scenario by returning predefined data. + # Mocks the CloudSQL enrichment handler to prevent actual database calls. + # This ensures that a cache hit scenario does not trigger any database + # interaction, raising an exception if an unexpected call occurs. actual = CloudSQLEnrichmentHandler.__call__ - CloudSQLEnrichmentHandler.__call__ = MagicMock(return_value=(beam.Row())) + CloudSQLEnrichmentHandler.__call__ = MagicMock( + side_effect=Exception("Database should not be called on a cache hit.")) # Run a second pipeline to verify cache is being used. with TestPipeline(is_integration_test=True) as test_pipeline: @@ -542,7 +544,6 @@ def setUpClass(cls): @classmethod def tearDownClass(cls): super().tearDownClass() - cls._db = None @unittest.skipUnless( From 7b6ba8f20e4b09993640262f3646dadfd5948f66 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sat, 19 Jul 2025 09:26:22 +0000 Subject: [PATCH 63/83] .github: trigger postcommit python --- .github/trigger_files/beam_PostCommit_Python.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/trigger_files/beam_PostCommit_Python.json b/.github/trigger_files/beam_PostCommit_Python.json index 9e1d1e1b80dd..c47b4b57e2ab 100644 --- a/.github/trigger_files/beam_PostCommit_Python.json +++ b/.github/trigger_files/beam_PostCommit_Python.json @@ -1,5 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run.", - "modification": 4 + "modification": 612 } From 84d5e4f91c9cd2ef1b2b5250391a72f46de1c7ed Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Mon, 21 Jul 2025 14:26:08 +0000 Subject: [PATCH 64/83] .github: trigger postcommit python --- .github/trigger_files/beam_PostCommit_Python.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/trigger_files/beam_PostCommit_Python.json b/.github/trigger_files/beam_PostCommit_Python.json index c47b4b57e2ab..02e8a2b932ab 100644 --- a/.github/trigger_files/beam_PostCommit_Python.json +++ b/.github/trigger_files/beam_PostCommit_Python.json @@ -1,5 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run.", - "modification": 612 + "modification": 617 } From 5462138da465d03235a1797797bfb7ad0fb3f841 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Mon, 21 Jul 2025 19:56:55 +0000 Subject: [PATCH 65/83] sdks/python: fix linting issues --- .../apache_beam/transforms/enrichment_handlers/cloudsql.py | 6 +++++- .../transforms/enrichment_handlers/cloudsql_it_test.py | 2 +- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py index 2110c51ccac0..d4371f01d145 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py @@ -316,7 +316,11 @@ def __enter__(self): self._engine = create_engine( url=self._connection_config.get_db_url(), creator=connector) - def _execute_query(self, query: str, is_batch: bool, **params): + def _execute_query( + self, + query: str, + is_batch: bool, + **params) -> Union[List[Dict[str, Any]], Dict[str, Any]]: connection = None try: connection = self._engine.connect() diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py index d556948d5104..db1a066cb7b2 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py @@ -90,7 +90,7 @@ def start_sql_db_container( sql_client_retries=3) -> Optional[SQLDBContainerInfo]: info = None for i in range(sql_client_retries): - sql_db_container = None + sql_db_container = DbContainer("") try: if database_type == DatabaseTypeAdapter.POSTGRESQL: user, password, db_id = "test", "test", "test" From a55ab60e31ffef4f1bfa66157e3de4a609622d09 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Tue, 22 Jul 2025 11:56:54 +0000 Subject: [PATCH 66/83] sdks/python: fix linting issues --- .../transforms/enrichment_handlers/cloudsql.py | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py index d4371f01d145..225b23ff6aa3 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py @@ -317,10 +317,8 @@ def __enter__(self): url=self._connection_config.get_db_url(), creator=connector) def _execute_query( - self, - query: str, - is_batch: bool, - **params) -> Union[List[Dict[str, Any]], Dict[str, Any]]: + self, query: str, is_batch: bool, + **params: Dict[str, Any]) -> Union[List[Dict[str, Any]], Dict[str, Any]]: connection = None try: connection = self._engine.connect() @@ -329,10 +327,13 @@ def _execute_query( result = connection.execute(text(query), **params) # Materialize results while transaction is active. if is_batch: - data = [row._asdict() for row in result] + data: List[Dict[str, Any]] = [row._asdict() for row in result] else: result_row = result.first() - data = result_row._asdict() if result_row else {} + if result_row: + data: Dict[str, Any] = result_row._asdict() + else: + data = {} # Explicitly commit the transaction. transaction.commit() return data From 42652ac93d33aa9f35f8f834e5df8581051caa13 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Tue, 22 Jul 2025 11:57:42 +0000 Subject: [PATCH 67/83] .github: trigger post commit python --- .github/trigger_files/beam_PostCommit_Python.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/trigger_files/beam_PostCommit_Python.json b/.github/trigger_files/beam_PostCommit_Python.json index 02e8a2b932ab..ca3300f988db 100644 --- a/.github/trigger_files/beam_PostCommit_Python.json +++ b/.github/trigger_files/beam_PostCommit_Python.json @@ -1,5 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run.", - "modification": 617 + "modification": 512 } From b77232f1d4873c062ba00bf671cfd5b174c4686e Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Tue, 22 Jul 2025 13:52:38 +0000 Subject: [PATCH 68/83] sdks/python: fix linting issues --- .../transforms/enrichment_handlers/cloudsql.py | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py index 225b23ff6aa3..a4c128921f75 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py @@ -326,14 +326,12 @@ def _execute_query( try: result = connection.execute(text(query), **params) # Materialize results while transaction is active. + data: Union[List[Dict[str, Any]], Dict[str, Any]] if is_batch: - data: List[Dict[str, Any]] = [row._asdict() for row in result] + data = [row._asdict() for row in result] else: result_row = result.first() - if result_row: - data: Dict[str, Any] = result_row._asdict() - else: - data = {} + data = result_row._asdict() if result_row else {} # Explicitly commit the transaction. transaction.commit() return data From ffd2f3824a1ead15157c14bd5ae921ee9a0118f9 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Tue, 22 Jul 2025 13:54:07 +0000 Subject: [PATCH 69/83] .github: trigger postcommit python --- .github/trigger_files/beam_PostCommit_Python.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/trigger_files/beam_PostCommit_Python.json b/.github/trigger_files/beam_PostCommit_Python.json index ca3300f988db..b8f3a0d36a72 100644 --- a/.github/trigger_files/beam_PostCommit_Python.json +++ b/.github/trigger_files/beam_PostCommit_Python.json @@ -1,5 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run.", - "modification": 512 + "modification": 128 } From 318a8bf27315935059ed9edd348a1c9dc582f828 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Tue, 22 Jul 2025 14:34:31 +0000 Subject: [PATCH 70/83] sdks/python: fix linting issues --- .../apache_beam/transforms/enrichment_handlers/cloudsql.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py index a4c128921f75..2c73f332e161 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py @@ -317,14 +317,14 @@ def __enter__(self): url=self._connection_config.get_db_url(), creator=connector) def _execute_query( - self, query: str, is_batch: bool, - **params: Dict[str, Any]) -> Union[List[Dict[str, Any]], Dict[str, Any]]: + self, query: str, + is_batch: bool) -> Union[List[Dict[str, Any]], Dict[str, Any]]: connection = None try: connection = self._engine.connect() transaction = connection.begin() try: - result = connection.execute(text(query), **params) + result = connection.execute(text(query)) # Materialize results while transaction is active. data: Union[List[Dict[str, Any]], Dict[str, Any]] if is_batch: From 2f21f7fb765cc91fe5cdf0ace5248685e14c9c07 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Tue, 22 Jul 2025 14:35:04 +0000 Subject: [PATCH 71/83] .github: trigger postcommit python --- .github/trigger_files/beam_PostCommit_Python.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/trigger_files/beam_PostCommit_Python.json b/.github/trigger_files/beam_PostCommit_Python.json index b8f3a0d36a72..637ef621e91c 100644 --- a/.github/trigger_files/beam_PostCommit_Python.json +++ b/.github/trigger_files/beam_PostCommit_Python.json @@ -1,5 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run.", - "modification": 128 + "modification": 64 } From b8f7e7a4d48ccfdae1b4f71f59a79d6a2d0b38f3 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Tue, 22 Jul 2025 15:33:48 +0000 Subject: [PATCH 72/83] sdks/python: fix linting issues --- .../apache_beam/transforms/enrichment_handlers/cloudsql.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py index 2c73f332e161..c1b25e73dd49 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py @@ -392,8 +392,8 @@ def _process_batch_request(self, requests: list[beam.Row]): # Formulate the query, execute it, and return a list of original requests # paired with their responses. query = raw_query.format(*values) - responses_dict = self._execute_query(query, is_batch=True) - for response in responses_dict: + responses: List[Dict[str, Any]] = self._execute_query(query, is_batch=True) + for response in responses: response_row = beam.Row(**response) response_key = self.create_row_key(response_row) if response_key in requests_map: @@ -419,7 +419,7 @@ def _process_single_request(self, request: beam.Row): "Make sure the values passed in `where_clause_fields` are " "the keys in the input `beam.Row`." + str(e)) query = self.query_template.format(*values) - response_dict = self._execute_query(query, is_batch=False) + response_dict: Dict[str, Any] = self._execute_query(query, is_batch=False) return request, beam.Row(**response_dict) def create_row_key(self, row: beam.Row): From 17cff20ec424edb29180b7dd612bdb0542800f3f Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Tue, 22 Jul 2025 15:34:00 +0000 Subject: [PATCH 73/83] .github: trigger postcommit python --- .github/trigger_files/beam_PostCommit_Python.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/trigger_files/beam_PostCommit_Python.json b/.github/trigger_files/beam_PostCommit_Python.json index 637ef621e91c..ed56f65ef50f 100644 --- a/.github/trigger_files/beam_PostCommit_Python.json +++ b/.github/trigger_files/beam_PostCommit_Python.json @@ -1,5 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run.", - "modification": 64 + "modification": 32 } From 7be26c1f65faa06e6f7679548e58cc8502a41408 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Tue, 22 Jul 2025 18:29:52 +0000 Subject: [PATCH 74/83] sdks/python: fix linting issues --- .../transforms/enrichment_handlers/cloudsql.py | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py index c1b25e73dd49..796534e61492 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py @@ -392,8 +392,9 @@ def _process_batch_request(self, requests: list[beam.Row]): # Formulate the query, execute it, and return a list of original requests # paired with their responses. query = raw_query.format(*values) - responses: List[Dict[str, Any]] = self._execute_query(query, is_batch=True) - for response in responses: + result: Union[List[Dict[str, Any]], Dict[str, Any]] = self._execute_query( + query, is_batch=True) + for response in result: response_row = beam.Row(**response) response_key = self.create_row_key(response_row) if response_key in requests_map: @@ -419,8 +420,9 @@ def _process_single_request(self, request: beam.Row): "Make sure the values passed in `where_clause_fields` are " "the keys in the input `beam.Row`." + str(e)) query = self.query_template.format(*values) - response_dict: Dict[str, Any] = self._execute_query(query, is_batch=False) - return request, beam.Row(**response_dict) + response: Union[List[Dict[str, Any]], Dict[str, Any]] = self._execute_query( + query, is_batch=False) + return request, beam.Row(**response) def create_row_key(self, row: beam.Row): if isinstance(self._query_config, TableFunctionQueryConfig): From 7fbeb5c5add1034aa40fcf70209e5b3620b963da Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 3 Aug 2025 17:58:14 +0000 Subject: [PATCH 75/83] sdks/python: fix SQL injection --- .../enrichment_handlers/cloudsql.py | 331 ++++++++++++---- .../enrichment_handlers/cloudsql_it_test.py | 12 +- .../enrichment_handlers/cloudsql_test.py | 361 +++++++++++++++++- 3 files changed, 608 insertions(+), 96 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py index 796534e61492..c3232ce1ac37 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py @@ -14,6 +14,7 @@ # See the License for the specific language governing permissions and # limitations under the License. # +import re from abc import ABC from abc import abstractmethod from collections.abc import Callable @@ -43,6 +44,7 @@ ConditionValueFn = Callable[[beam.Row], list[Any]] + @dataclass class CustomQueryConfig: """Configuration for using a custom query function.""" @@ -72,6 +74,8 @@ def __post_init__(self): "where_clause_fields") + + @dataclass class TableFunctionQueryConfig: """Configuration for using table name, where clause, and a value function.""" @@ -90,6 +94,8 @@ def __post_init__(self): "TableFunctionQueryConfig must provide " + "where_clause_value_fn") + + class DatabaseTypeAdapter(Enum): POSTGRESQL = "pg8000" MYSQL = "pymysql" @@ -203,18 +209,19 @@ def get_connector_handler(self) -> Callable[[], DBAPIConnection]: be properly closed by the caller when no longer needed. """ if self.db_adapter == DatabaseTypeAdapter.POSTGRESQL: - sql_connector = lambda: pg8000.connect( + return lambda: pg8000.connect( host=self.host, port=self.port, database=self.db_id, user=self.user, password=self.password, **self.connect_kwargs) elif self.db_adapter == DatabaseTypeAdapter.MYSQL: - sql_connector = lambda: pymysql.connect( + return lambda: pymysql.connect( host=self.host, port=self.port, database=self.db_id, user=self.user, password=self.password, **self.connect_kwargs) elif self.db_adapter == DatabaseTypeAdapter.SQLSERVER: - sql_connector = lambda: pytds.connect( + return lambda: pytds.connect( dsn=self.host, port=self.port, database=self.db_id, user=self.user, password=self.password, **self.connect_kwargs) - return sql_connector + else: + raise ValueError(f"Unsupported database adapter: {self.db_adapter}") def get_db_url(self) -> str: return self.db_adapter.to_sqlalchemy_dialect() + "://" @@ -264,7 +271,7 @@ def __init__( instance_connection_uri="apache-beam-testing:us-central1:itests", user='postgres', password= os.getenv("CLOUDSQL_PG_PASSWORD")) - query_config=TableFieldsQueryConfig('my_table',"id = '{}'",['id']), + query_config=TableFieldsQueryConfig('my_table',"id = :param0",['id']), cloudsql_handler = CloudSQLEnrichmentHandler( connection_config=connection_config, query_config=query_config, @@ -316,15 +323,68 @@ def __enter__(self): self._engine = create_engine( url=self._connection_config.get_db_url(), creator=connector) + def __call__( + self, request: Union[beam.Row, list[beam.Row]], *_args, **_kwargs): + """Handle requests by delegating to single or batch processing.""" + if isinstance(request, list): + return self._process_batch_request(request) + else: + return self._process_single_request(request) + + def _process_single_request(self, request: beam.Row): + """Process a single request and return with its response.""" + response: Union[List[Dict[str, Any]], Dict[str, Any]] + if isinstance(self._query_config, CustomQueryConfig): + query = self._query_config.query_fn(request) + response = self._execute_query(query, is_batch=False) + else: + values = self._extract_values_from_request(request) + param_dict = self._build_single_param_dict(values) + response = self._execute_query( + self.query_template, params=param_dict, is_batch=False) + return request, beam.Row(**response) + + def _process_batch_request(self, requests: list[beam.Row]): + """Process batch requests and match responses to original requests.""" + values, responses = [], [] + requests_map: dict[Any, Any] = {} + batch_size = len(requests) + + # Build the appropriate query (single or batched). + raw_query = self._build_batch_query(requests, batch_size) + + # Extract where_clause_fields values and map the generated request key to + # the original request object.. + for req in requests: + current_values = self._extract_values_from_request(req) + values.extend(current_values) + requests_map[self.create_row_key(req)] = req + + # Build named parameters dictionary for parameterized query. + param_dict = self._build_parameters_dict(requests, batch_size) + + # Execute the parameterized query with validated parameters. + result: Union[List[Dict[str, Any]], Dict[str, Any]] = self._execute_query( + raw_query, params=param_dict, is_batch=True) + for response in result: + response_row = beam.Row(**response) + response_key = self.create_row_key(response_row) + if response_key in requests_map: + responses.append((requests_map[response_key], response_row)) + return responses + def _execute_query( - self, query: str, - is_batch: bool) -> Union[List[Dict[str, Any]], Dict[str, Any]]: + self, query: str, params: Optional[dict] = None, + is_batch: bool = False) -> Union[List[Dict[str, Any]], Dict[str, Any]]: connection = None try: connection = self._engine.connect() transaction = connection.begin() try: - result = connection.execute(text(query)) + if params: + result = connection.execute(text(query), params) + else: + result = connection.execute(text(query)) # Materialize results while transaction is active. data: Union[List[Dict[str, Any]], Dict[str, Any]] if is_batch: @@ -346,83 +406,186 @@ def _execute_query( if connection: connection.close() - def __call__(self, request: Union[beam.Row, list[beam.Row]], *args, **kwargs): - """Handle requests by delegating to single or batch processing.""" - if isinstance(request, list): - return self._process_batch_request(request) - else: - return self._process_single_request(request) + def _build_batch_query( + self, requests: list[beam.Row], batch_size: int) -> str: + """Build batched query with unique parameter names for multiple requests. - def _process_batch_request(self, requests: list[beam.Row]): - """Process batch requests and match responses to original requests.""" - values, responses = [], [] - requests_map: dict[Any, Any] = {} - batch_size = len(requests) - raw_query = self.query_template + This method extracts parameter placeholders from the where_clause_template + using regex and creates unique parameter names for each batch item. The + parameter names in the template can be any valid identifiers (e.g., :id, + :param_0, :user_name) and don't need to match field names exactly. - # For multiple requests in the batch, combine the WHERE clause conditions - # using 'OR' and update the query template to handle all requests. + For batch queries, placeholders are replaced with unique names like + :batch_0_id, :batch_1_param_0, etc., based on the actual parameter names + found in the template. + + Args: + requests: List of beam.Row requests to process + batch_size: Number of requests in the batch + + Returns: + SQL query string with batched WHERE clauses using unique parameter names + """ + # Single request - return original query. + if batch_size <= 1: + return self.query_template + + # Only batch table-based query configs. table_query_configs = (TableFieldsQueryConfig, TableFunctionQueryConfig) - if batch_size > 1 and isinstance(self._query_config, table_query_configs): - where_clause_template_batched = ' OR '.join( - [fr'({self._query_config.where_clause_template})'] * batch_size) - raw_query = self.query_template.replace( - self._query_config.where_clause_template, - where_clause_template_batched) + if not isinstance(self._query_config, table_query_configs): + return self.query_template - # Extract where_clause_fields values and map the generated request key to - # the original request object. - for req in requests: - request_dict = req._asdict() - try: - if isinstance(self._query_config, TableFunctionQueryConfig): - current_values = self._query_config.where_clause_value_fn(req) - elif isinstance(self._query_config, TableFieldsQueryConfig): - current_values = [ - request_dict[field] - for field in self._query_config.where_clause_fields - ] - except KeyError as e: - raise KeyError( - "Make sure the values passed in `where_clause_fields` are " - " thekeys in the input `beam.Row`." + str(e)) - values.extend(current_values) - requests_map[self.create_row_key(req)] = req + # Build batched WHERE clauses. + where_clauses = [self._create_batch_clause(i) for i in range(batch_size)] - # Formulate the query, execute it, and return a list of original requests - # paired with their responses. - query = raw_query.format(*values) - result: Union[List[Dict[str, Any]], Dict[str, Any]] = self._execute_query( - query, is_batch=True) - for response in result: - response_row = beam.Row(**response) - response_key = self.create_row_key(response_row) - if response_key in requests_map: - responses.append((requests_map[response_key], response_row)) - return responses + # Combine clauses and update query. + where_clause_batched = ' OR '.join(where_clauses) + return self.query_template.replace( + self._query_config.where_clause_template, + where_clause_batched) - def _process_single_request(self, request: beam.Row): - """Process a single request and return with its response.""" - request_dict = request._asdict() - if isinstance(self._query_config, CustomQueryConfig): - query = self._query_config.query_fn(request) - else: - try: - if isinstance(self._query_config, TableFunctionQueryConfig): - values = self._query_config.where_clause_value_fn(request) - elif isinstance(self._query_config, TableFieldsQueryConfig): - values = [ - request_dict[field] - for field in self._query_config.where_clause_fields - ] - except KeyError as e: - raise KeyError( - "Make sure the values passed in `where_clause_fields` are " - "the keys in the input `beam.Row`." + str(e)) - query = self.query_template.format(*values) - response: Union[List[Dict[str, Any]], Dict[str, Any]] = self._execute_query( - query, is_batch=False) - return request, beam.Row(**response) + def _create_batch_clause(self, batch_index: int) -> str: + """Create a WHERE clause for a single batch item with unique parameter + names.""" + clause = self._query_config.where_clause_template + + # Extract parameter names from the template using regex. + param_names = self._extract_parameter_names( + self._query_config.where_clause_template) + for param_name in param_names: + old_param = f':{param_name}' + new_param = f':batch_{batch_index}_{param_name}' + clause = clause.replace(old_param, new_param) + + return f'({clause})' + + def _build_parameters_dict( + self, requests: list[beam.Row], batch_size: int) -> dict: + """Build named parameters dictionary for parameterized queries. + + Args: + requests: List of beam.Row requests to process + batch_size: Number of requests in the batch + + Returns: + Dictionary mapping parameter names to validated values + """ + param_dict = {} + for i, req in enumerate(requests): + current_values = self._extract_values_from_request(req) + + # For batched queries, use unique parameter names per batch item. + if batch_size > 1: + # Extract parameter names from the template using regex. + param_names = self._extract_parameter_names( + self._query_config.where_clause_template) + for param_name, val in zip(param_names, current_values): + param_dict[f'batch_{i}_{param_name}'] = val + else: + # For single request, use the helper function. + single_param_dict = self._build_single_param_dict(current_values) + param_dict.update(single_param_dict) + + return param_dict + + def _build_single_param_dict(self, values: list[Any]) -> dict[str, Any]: + """Build parameter dictionary for single request processing. + + Args: + values: List of parameter values + + Returns: + Dictionary mapping parameter names to values + """ + if isinstance(self._query_config, TableFieldsQueryConfig): + return { + field_name: val for field_name, val in zip( + self._query_config.where_clause_fields, values)} + else: # TableFunctionQueryConfig. + _, param_dict = self._get_unique_template_and_params( + self._query_config.where_clause_template, values) + return param_dict + + def _get_unique_template_and_params( + self, template: str, values: list[Any]) -> tuple[str, dict[str, Any]]: + """Generate unique binding parameter names for duplicate templates. + + Args: + template: SQL template with potentially duplicate binding parameter names + values: List of parameter values + + Returns: + Tuple of (updated_template, param_dict) with unique binding names. + """ + param_names = self._extract_parameter_names(template) + unique_param_names = [ + f"{param_name}_{i}" if param_names.count(param_name) > 1 else param_name + for i, param_name in enumerate(param_names) + ] + + # Update template by replacing each parameter occurrence in order. + updated_template = template + param_positions = [] + + # Find all parameter positions. + for match in re.finditer(r':(\w+)', template): + param_positions.append((match.start(), match.end(), match.group(1))) + + # Replace parameters from right to left to avoid position shifts. + for i in reversed(range(len(param_positions))): + start, end, _ = param_positions[i] + unique_name = unique_param_names[i] + updated_template = (updated_template[:start] + + f':{unique_name}' + + updated_template[end:]) + + # Build parameter dictionary. + param_dict = { + unique_name: val for unique_name, val in zip(unique_param_names, values) + } + + return updated_template, param_dict + + def _extract_values_from_request(self, request: beam.Row) -> list[Any]: + """Extract parameter values from a request based on query configuration. + + Args: + request: The beam.Row request to extract values from + + Returns: + List of parameter values + + Raises: + KeyError: If required fields are missing from the request + """ + try: + if isinstance(self._query_config, TableFunctionQueryConfig): + return [ + val for val in self._query_config.where_clause_value_fn(request) + ] + elif isinstance(self._query_config, TableFieldsQueryConfig): + request_dict = request._asdict() + return [ + request_dict[field] + for field in self._query_config.where_clause_fields + ] + else: + raise ValueError("Unsupported query configuration type") + except KeyError as e: + raise KeyError( + "Make sure the values passed in `where_clause_fields` are " + "the keys in the input `beam.Row`." + str(e)) + + def _extract_parameter_names(self, template: str) -> list[str]: + """Extract parameter names from a SQL template string. + + Args: + template: SQL template string with named parameters (e.g., "id = :id") + + Returns: + List of parameter names found in the template (e.g., ["id"]) + """ + return re.findall(r':(\w+)', template) def create_row_key(self, row: beam.Row): if isinstance(self._query_config, TableFunctionQueryConfig): @@ -436,10 +599,6 @@ def create_row_key(self, row: beam.Row): raise ValueError( "Either where_clause_fields or where_clause_value_fn must be specified") - def __exit__(self, exc_type, exc_val, exc_tb): - self._engine.dispose(close=True) - self._engine = None - def get_cache_key(self, request: Union[beam.Row, list[beam.Row]]): if isinstance(self._query_config, CustomQueryConfig): raise NotImplementedError( @@ -471,7 +630,7 @@ def get_cache_key(self, request: Union[beam.Row, list[beam.Row]]): try: if isinstance(self._query_config, TableFunctionQueryConfig): current_values = self._query_config.where_clause_value_fn(request) - else: # TableFieldsQueryConfig + else: # TableFieldsQueryConfig. current_values = [ req_dict[field] for field in self._query_config.where_clause_fields @@ -484,6 +643,10 @@ def get_cache_key(self, request: Union[beam.Row, list[beam.Row]]): "keys in the input `beam.Row`." + str(e)) return cache_key + def __exit__(self, _exc_type, _exc_val, _exc_tb): + self._engine.dispose(close=True) + self._engine = None + def batch_elements_kwargs(self) -> Mapping[str, Any]: """Returns a kwargs suitable for `beam.BatchElements`.""" return self._batching_kwargs diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py index db1a066cb7b2..3d9cd18151b6 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py @@ -320,7 +320,7 @@ def test_sql_enrichment(self): query_config = TableFieldsQueryConfig( table_id=self._table_id, - where_clause_template="id = {}", + where_clause_template="id = :id", where_clause_fields=fields) handler = CloudSQLEnrichmentHandler( @@ -348,7 +348,7 @@ def test_sql_enrichment_batched(self): query_config = TableFieldsQueryConfig( table_id=self._table_id, - where_clause_template="id = {}", + where_clause_template="id = :id", where_clause_fields=fields) handler = CloudSQLEnrichmentHandler( @@ -375,7 +375,7 @@ def test_sql_enrichment_batched_multiple_fields(self): query_config = TableFieldsQueryConfig( table_id=self._table_id, - where_clause_template="id = {} AND distribution_center_id = {}", + where_clause_template="id = :id AND distribution_center_id = :param_1", where_clause_fields=fields) handler = CloudSQLEnrichmentHandler( @@ -421,7 +421,7 @@ def test_sql_enrichment_with_condition_value_fn(self): query_config = TableFunctionQueryConfig( table_id=self._table_id, - where_clause_template="id = {}", + where_clause_template="id = :param_0", where_clause_value_fn=where_clause_value_fn) handler = CloudSQLEnrichmentHandler( @@ -442,7 +442,7 @@ def test_sql_enrichment_on_non_existent_table(self): query_config = TableFunctionQueryConfig( table_id=self._table_id, - where_clause_template="id = {}", + where_clause_template="id = :id", where_clause_value_fn=where_clause_value_fn) handler = CloudSQLEnrichmentHandler( @@ -473,7 +473,7 @@ def test_sql_enrichment_with_redis(self): query_config = TableFunctionQueryConfig( table_id=self._table_id, - where_clause_template="id = {}", + where_clause_template="id = :param_0", where_clause_value_fn=where_clause_value_fn) handler = CloudSQLEnrichmentHandler( diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_test.py index 00223db707b7..43453b97abac 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_test.py @@ -149,7 +149,7 @@ def test_valid_query_configs(self): # Valid TableFieldsQueryConfig. table_fields_config = TableFieldsQueryConfig( table_id="my_table", - where_clause_template="id = '{}'", + where_clause_template="id = :id", where_clause_fields=["id"]) connection_config = ExternalSQLDBConnectionConfig( @@ -167,12 +167,12 @@ def test_valid_query_configs(self): max_batch_size=10) self.assertEqual( - handler1.query_template, "SELECT * FROM my_table WHERE id = '{}'") + handler1.query_template, "SELECT * FROM my_table WHERE id = :id") # Valid TableFunctionQueryConfig. table_function_config = TableFunctionQueryConfig( table_id="my_table", - where_clause_template="id = '{}'", + where_clause_template="id = :id", where_clause_value_fn=where_clause_value_fn) handler2 = CloudSQLEnrichmentHandler( @@ -182,7 +182,7 @@ def test_valid_query_configs(self): max_batch_size=10) self.assertEqual( - handler2.query_template, "SELECT * FROM my_table WHERE id = '{}'") + handler2.query_template, "SELECT * FROM my_table WHERE id = :id") # Valid CustomQueryConfig. custom_config = CustomQueryConfig(query_fn=query_fn) @@ -208,14 +208,363 @@ def test_custom_query_config_cache_key_error(self): handler = CloudSQLEnrichmentHandler( connection_config=connection_config, query_config=custom_config) - # Create a dummy request + # Create a dummy request. import apache_beam as beam request = beam.Row(id=1) - # Verify that get_cache_key raises NotImplementedError + # Verify that get_cache_key raises NotImplementedError. with self.assertRaises(NotImplementedError): handler.get_cache_key(request) + def test_extract_parameter_names(self): + """Test parameter extraction from SQL templates.""" + connection_config = ExternalSQLDBConnectionConfig( + db_adapter=DatabaseTypeAdapter.POSTGRESQL, + host='localhost', + port=5432, + user='user', + password='password', + db_id='db') + + config = TableFieldsQueryConfig( + table_id="users", + where_clause_template="id = :id", + where_clause_fields=["id"]) + + handler = CloudSQLEnrichmentHandler( + connection_config=connection_config, query_config=config) + + # Test simple parameter extraction. + self.assertEqual( + handler._extract_parameter_names("id = :id"), ["id"]) + + # Test multiple parameters. + self.assertEqual( + handler._extract_parameter_names("id = :id AND name = :name"), + ["id", "name"]) + + # Test no parameters. + self.assertEqual( + handler._extract_parameter_names("SELECT * FROM users"), []) + + # Test complex query. + self.assertEqual( + handler._extract_parameter_names( + "age > :min_age AND city = :city AND status = :status"), + ["min_age", "city", "status"]) + + def test_build_single_param_dict(self): + """Test building parameter dictionaries for single requests.""" + connection_config = ExternalSQLDBConnectionConfig( + db_adapter=DatabaseTypeAdapter.POSTGRESQL, + host='localhost', + port=5432, + user='user', + password='password', + db_id='db') + + # Test TableFieldsQueryConfig. + table_config = TableFieldsQueryConfig( + table_id="users", + where_clause_template="id = :id AND name = :name", + where_clause_fields=["id", "name"]) + + handler1 = CloudSQLEnrichmentHandler( + connection_config=connection_config, query_config=table_config) + + result1 = handler1._build_single_param_dict([123, "John"]) + self.assertEqual(result1, {"id": 123, "name": "John"}) + + # Test TableFunctionQueryConfig. + table_func_config = TableFunctionQueryConfig( + table_id="users", + where_clause_template="age > :min_age AND city = :city", + where_clause_value_fn=lambda row: [row.min_age, row.city]) + + handler2 = CloudSQLEnrichmentHandler( + connection_config=connection_config, query_config=table_func_config) + + result2 = handler2._build_single_param_dict([3, "NYC"]) + self.assertEqual(result2, {"min_age": 3, "city": "NYC"}) + + def test_extract_values_from_request(self): + """Test extracting values from requests based on query configuration.""" + connection_config = ExternalSQLDBConnectionConfig( + db_adapter=DatabaseTypeAdapter.POSTGRESQL, + host='localhost', + port=5432, + user='user', + password='password', + db_id='db') + + # Test TableFieldsQueryConfig. + table_config = TableFieldsQueryConfig( + table_id="users", + where_clause_template="id = :id AND name = :name", + where_clause_fields=["id", "name"]) + + handler1 = CloudSQLEnrichmentHandler( + connection_config=connection_config, query_config=table_config) + + import apache_beam as beam + request1 = beam.Row(id=123, name="John", age=30) + result1 = handler1._extract_values_from_request(request1) + self.assertEqual(result1, [123, "John"]) + + # Test TableFunctionQueryConfig. + def test_value_fn(row): + return [row.age, row.city] + + table_func_config = TableFunctionQueryConfig( + table_id="users", + where_clause_template="age > :min_age AND city = :city", + where_clause_value_fn=test_value_fn) + + handler2 = CloudSQLEnrichmentHandler( + connection_config=connection_config, query_config=table_func_config) + + request2 = beam.Row(age=25, city="NYC", name="Jane") + result2 = handler2._extract_values_from_request(request2) + self.assertEqual(result2, [25, "NYC"]) + + # Test missing field error. + request3 = beam.Row(age=30) # Missing name field. + with self.assertRaises(KeyError) as context: + handler1._extract_values_from_request(request3) + self.assertIn("where_clause_fields", str(context.exception)) + + def test_build_batch_query_single_request(self): + """Test batch query building with single request returns original query.""" + connection_config = ExternalSQLDBConnectionConfig( + db_adapter=DatabaseTypeAdapter.POSTGRESQL, + host='localhost', + port=5432, + user='user', + password='password', + db_id='db') + + config = TableFieldsQueryConfig( + table_id="users", + where_clause_template="id = :id", + where_clause_fields=["id"]) + + handler = CloudSQLEnrichmentHandler( + connection_config=connection_config, query_config=config) + + import apache_beam as beam + requests = [beam.Row(id=1)] + + # Single request should return original query template. + result = handler._build_batch_query(requests, batch_size=1) + self.assertEqual(result, "SELECT * FROM users WHERE id = :id") + + def test_build_batch_query_multiple_requests(self): + """Test batch query building with multiple requests creates OR clauses.""" + connection_config = ExternalSQLDBConnectionConfig( + db_adapter=DatabaseTypeAdapter.POSTGRESQL, + host='localhost', + port=5432, + user='user', + password='password', + db_id='db') + + config = TableFieldsQueryConfig( + table_id="users", + where_clause_template="id = :id AND name = :name", + where_clause_fields=["id", "name"]) + + handler = CloudSQLEnrichmentHandler( + connection_config=connection_config, query_config=config) + + import apache_beam as beam + requests = [beam.Row(id=1, name="Alice"), beam.Row(id=2, name="Bob")] + + result = handler._build_batch_query(requests, batch_size=2) + expected = ("SELECT * FROM users WHERE " + "(id = :batch_0_id AND name = :batch_0_name) OR " + "(id = :batch_1_id AND name = :batch_1_name)") + self.assertEqual(result, expected) + + def test_build_parameters_dict_batch(self): + """Test parameter dictionary building for batch requests.""" + connection_config = ExternalSQLDBConnectionConfig( + db_adapter=DatabaseTypeAdapter.POSTGRESQL, + host='localhost', + port=5432, + user='user', + password='password', + db_id='db') + + config = TableFieldsQueryConfig( + table_id="users", + where_clause_template="id = :id", + where_clause_fields=["id"]) + + handler = CloudSQLEnrichmentHandler( + connection_config=connection_config, query_config=config) + + import apache_beam as beam + requests = [beam.Row(id=1), beam.Row(id=2)] + + result = handler._build_parameters_dict(requests, batch_size=2) + expected = {"batch_0_id": 1, "batch_1_id": 2} + self.assertEqual(result, expected) + + def test_create_batch_clause(self): + """Test batch clause creation with unique parameter names.""" + connection_config = ExternalSQLDBConnectionConfig( + db_adapter=DatabaseTypeAdapter.POSTGRESQL, + host='localhost', + port=5432, + user='user', + password='password', + db_id='db') + + config = TableFieldsQueryConfig( + table_id="users", + where_clause_template="id = :id AND name = :name", + where_clause_fields=["id", "name"]) + + handler = CloudSQLEnrichmentHandler( + connection_config=connection_config, query_config=config) + + result = handler._create_batch_clause(batch_index=0) + expected = "(id = :batch_0_id AND name = :batch_0_name)" + self.assertEqual(result, expected) + + result2 = handler._create_batch_clause(batch_index=1) + expected2 = "(id = :batch_1_id AND name = :batch_1_name)" + self.assertEqual(result2, expected2) + + def test_security_parameter_extraction_edge_cases(self): + """Test parameter extraction with edge cases and security issues.""" + connection_config = ExternalSQLDBConnectionConfig( + db_adapter=DatabaseTypeAdapter.POSTGRESQL, + host='localhost', + port=5432, + user='user', + password='password', + db_id='db') + + config = TableFieldsQueryConfig( + table_id="users", + where_clause_template="id = :id", + where_clause_fields=["id"]) + + handler = CloudSQLEnrichmentHandler( + connection_config=connection_config, query_config=config) + + # Test empty template. + self.assertEqual(handler._extract_parameter_names(""), []) + + # Test template with no parameters. + self.assertEqual( + handler._extract_parameter_names("SELECT * FROM users"), []) + + # Test template with malformed parameters (should not match). + self.assertEqual( + handler._extract_parameter_names("id = :"), []) + + # Test template with numeric parameter names. + self.assertEqual( + handler._extract_parameter_names("col = :param_123"), ["param_123"]) + + # Test template with underscore parameter names. + self.assertEqual( + handler._extract_parameter_names("col = :user_id"), ["user_id"]) + + # Test duplicate parameter names. + result = handler._extract_parameter_names("id = :id OR id = :id") + # Note: re.findall returns all matches, so we'd get ["id", "id"]. + self.assertEqual(result, ["id", "id"]) + + def test_build_single_param_dict_with_generic_names(self): + """Test parameter dict building with generic names.""" + connection_config = ExternalSQLDBConnectionConfig( + db_adapter=DatabaseTypeAdapter.POSTGRESQL, + host='localhost', + port=5432, + user='user', + password='password', + db_id='db') + + # Test TableFunctionQueryConfig with generic parameter names. + config = TableFunctionQueryConfig( + table_id="users", + where_clause_template="id = :param_0 AND name = :param_1", + where_clause_value_fn=lambda row: [row.id, row.name]) + + handler = CloudSQLEnrichmentHandler( + connection_config=connection_config, query_config=config) + + result = handler._build_single_param_dict([123, "John"]) + expected = {"param_0": 123, "param_1": "John"} + self.assertEqual(result, expected) + + def test_duplicate_binding_parameter_names_handling(self): + """Test handling of duplicate binding parameter names.""" + connection_config = ExternalSQLDBConnectionConfig( + db_adapter=DatabaseTypeAdapter.POSTGRESQL, + host='localhost', + port=5432, + user='user', + password='password', + db_id='db') + + # Test TableFunctionQueryConfig with duplicate parameter names. + config = TableFunctionQueryConfig( + table_id="users", + where_clause_template="id = :param AND name = :param", + where_clause_value_fn=lambda row: [row.id, row.name]) + + handler = CloudSQLEnrichmentHandler( + connection_config=connection_config, query_config=config) + + # Test that duplicate parameter names are made unique. + result = handler._build_single_param_dict([123, "John"]) + expected = {"param_0": 123, "param_1": "John"} + self.assertEqual(result, expected) + + # Test the helper method for template and params. + template = "id = :param AND name = :param" + values = [123, "John"] + updated_template, param_dict = handler._get_unique_template_and_params( + template, values) + + expected_template = "id = :param_0 AND name = :param_1" + expected_params = {"param_0": 123, "param_1": "John"} + self.assertEqual(updated_template, expected_template) + self.assertEqual(param_dict, expected_params) + + def test_unsupported_query_config_type_error(self): + """Test that unsupported query config types raise ValueError.""" + connection_config = ExternalSQLDBConnectionConfig( + db_adapter=DatabaseTypeAdapter.POSTGRESQL, + host='localhost', + port=5432, + user='user', + password='password', + db_id='db') + + config = TableFieldsQueryConfig( + table_id="users", + where_clause_template="id = :id", + where_clause_fields=["id"]) + + handler = CloudSQLEnrichmentHandler( + connection_config=connection_config, query_config=config) + + # Temporarily replace the config with an unsupported type. + handler._query_config = "unsupported_type" + + import apache_beam as beam + request = beam.Row(id=1) + + with self.assertRaises(ValueError) as context: + handler._extract_values_from_request(request) + self.assertIn( + "Unsupported query configuration type", str(context.exception)) + if __name__ == '__main__': unittest.main() From f9d00051c0a4dfb8070596d56a20db4bc4c95565 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 3 Aug 2025 20:16:05 +0000 Subject: [PATCH 76/83] CHANGES.md: update release notes --- CHANGES.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/CHANGES.md b/CHANGES.md index 206ac3ba11ad..724357a2077c 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -72,6 +72,10 @@ ## New Features / Improvements * X feature added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). +* Google CloudSQL enrichment handler added (Python) ([#34398](https://github.com/apache/beam/pull/34398)). + Beam now supports data enrichment capabilities using SQL databases, with built-in support for: + - Managed PostgreSQL, MySQL, and Microsoft SQL Server instances on CloudSQL + - Unmanaged SQL database instances not hosted on CloudSQL (e.g., self-hosted or on-premises databases) ## Breaking Changes From 4c69da0c372f748b9aad5270681c4d1d6fd178be Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 3 Aug 2025 22:00:09 +0000 Subject: [PATCH 77/83] sdks/python: fix formatting issues --- .../enrichment_handlers/cloudsql.py | 26 +++++++++---------- .../enrichment_handlers/cloudsql_test.py | 13 +++++----- 2 files changed, 18 insertions(+), 21 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py index c3232ce1ac37..c5138ce7a777 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py @@ -44,7 +44,6 @@ ConditionValueFn = Callable[[beam.Row], list[Any]] - @dataclass class CustomQueryConfig: """Configuration for using a custom query function.""" @@ -74,8 +73,6 @@ def __post_init__(self): "where_clause_fields") - - @dataclass class TableFunctionQueryConfig: """Configuration for using table name, where clause, and a value function.""" @@ -94,8 +91,6 @@ def __post_init__(self): "TableFunctionQueryConfig must provide " + "where_clause_value_fn") - - class DatabaseTypeAdapter(Enum): POSTGRESQL = "pg8000" MYSQL = "pymysql" @@ -374,7 +369,9 @@ def _process_batch_request(self, requests: list[beam.Row]): return responses def _execute_query( - self, query: str, params: Optional[dict] = None, + self, + query: str, + params: Optional[dict] = None, is_batch: bool = False) -> Union[List[Dict[str, Any]], Dict[str, Any]]: connection = None try: @@ -441,8 +438,7 @@ def _build_batch_query( # Combine clauses and update query. where_clause_batched = ' OR '.join(where_clauses) return self.query_template.replace( - self._query_config.where_clause_template, - where_clause_batched) + self._query_config.where_clause_template, where_clause_batched) def _create_batch_clause(self, batch_index: int) -> str: """Create a WHERE clause for a single batch item with unique parameter @@ -499,8 +495,10 @@ def _build_single_param_dict(self, values: list[Any]) -> dict[str, Any]: """ if isinstance(self._query_config, TableFieldsQueryConfig): return { - field_name: val for field_name, val in zip( - self._query_config.where_clause_fields, values)} + field_name: val + for field_name, val in zip( + self._query_config.where_clause_fields, values) + } else: # TableFunctionQueryConfig. _, param_dict = self._get_unique_template_and_params( self._query_config.where_clause_template, values) @@ -535,13 +533,13 @@ def _get_unique_template_and_params( for i in reversed(range(len(param_positions))): start, end, _ = param_positions[i] unique_name = unique_param_names[i] - updated_template = (updated_template[:start] + - f':{unique_name}' + - updated_template[end:]) + updated_template = ( + updated_template[:start] + f':{unique_name}' + updated_template[end:]) # Build parameter dictionary. param_dict = { - unique_name: val for unique_name, val in zip(unique_param_names, values) + unique_name: val + for unique_name, val in zip(unique_param_names, values) } return updated_template, param_dict diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_test.py index 43453b97abac..99823f6d89a6 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_test.py @@ -235,8 +235,7 @@ def test_extract_parameter_names(self): connection_config=connection_config, query_config=config) # Test simple parameter extraction. - self.assertEqual( - handler._extract_parameter_names("id = :id"), ["id"]) + self.assertEqual(handler._extract_parameter_names("id = :id"), ["id"]) # Test multiple parameters. self.assertEqual( @@ -380,9 +379,10 @@ def test_build_batch_query_multiple_requests(self): requests = [beam.Row(id=1, name="Alice"), beam.Row(id=2, name="Bob")] result = handler._build_batch_query(requests, batch_size=2) - expected = ("SELECT * FROM users WHERE " - "(id = :batch_0_id AND name = :batch_0_name) OR " - "(id = :batch_1_id AND name = :batch_1_name)") + expected = ( + "SELECT * FROM users WHERE " + "(id = :batch_0_id AND name = :batch_0_name) OR " + "(id = :batch_1_id AND name = :batch_1_name)") self.assertEqual(result, expected) def test_build_parameters_dict_batch(self): @@ -462,8 +462,7 @@ def test_security_parameter_extraction_edge_cases(self): handler._extract_parameter_names("SELECT * FROM users"), []) # Test template with malformed parameters (should not match). - self.assertEqual( - handler._extract_parameter_names("id = :"), []) + self.assertEqual(handler._extract_parameter_names("id = :"), []) # Test template with numeric parameter names. self.assertEqual( From d8c4ae1c130a634ca0352ee6fdb21826b75d020a Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 3 Aug 2025 22:15:22 +0000 Subject: [PATCH 78/83] sdks/python: fix linting issues --- .../transforms/enrichment_handlers/cloudsql.py | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py index c5138ce7a777..3e06b112a746 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py @@ -337,7 +337,7 @@ def _process_single_request(self, request: beam.Row): param_dict = self._build_single_param_dict(values) response = self._execute_query( self.query_template, params=param_dict, is_batch=False) - return request, beam.Row(**response) + return request, beam.Row(**response) # type: ignore[arg-type] def _process_batch_request(self, requests: list[beam.Row]): """Process batch requests and match responses to original requests.""" @@ -362,7 +362,7 @@ def _process_batch_request(self, requests: list[beam.Row]): result: Union[List[Dict[str, Any]], Dict[str, Any]] = self._execute_query( raw_query, params=param_dict, is_batch=True) for response in result: - response_row = beam.Row(**response) + response_row = beam.Row(**response) # type: ignore[arg-type] response_key = self.create_row_key(response_row) if response_key in requests_map: responses.append((requests_map[response_key], response_row)) @@ -437,12 +437,17 @@ def _build_batch_query( # Combine clauses and update query. where_clause_batched = ' OR '.join(where_clauses) + # We know this is a table-based config from the check above. + assert isinstance(self._query_config, table_query_configs) return self.query_template.replace( self._query_config.where_clause_template, where_clause_batched) def _create_batch_clause(self, batch_index: int) -> str: """Create a WHERE clause for a single batch item with unique parameter names.""" + # This method is only called for table-based query configs + table_query_configs = (TableFieldsQueryConfig, TableFunctionQueryConfig) + assert isinstance(self._query_config, table_query_configs) clause = self._query_config.where_clause_template # Extract parameter names from the template using regex. @@ -473,6 +478,9 @@ def _build_parameters_dict( # For batched queries, use unique parameter names per batch item. if batch_size > 1: # Extract parameter names from the template using regex. + # Batching is only used with table-based query configs + table_query_configs = (TableFieldsQueryConfig, TableFunctionQueryConfig) + assert isinstance(self._query_config, table_query_configs) param_names = self._extract_parameter_names( self._query_config.where_clause_template) for param_name, val in zip(param_names, current_values): @@ -500,6 +508,7 @@ def _build_single_param_dict(self, values: list[Any]) -> dict[str, Any]: self._query_config.where_clause_fields, values) } else: # TableFunctionQueryConfig. + assert isinstance(self._query_config, TableFunctionQueryConfig) _, param_dict = self._get_unique_template_and_params( self._query_config.where_clause_template, values) return param_dict From 4da4af16189e38c2272f3ba90b71add7d192ff5c Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 3 Aug 2025 23:52:24 +0000 Subject: [PATCH 79/83] sdks/python: update py image requirements --- .../py310/base_image_requirements.txt | 34 +++---- .../container/py310/ml_image_requirements.txt | 40 ++++---- .../py311/base_image_requirements.txt | 36 +++---- .../container/py311/ml_image_requirements.txt | 42 ++++----- .../py312/base_image_requirements.txt | 36 +++---- .../container/py312/ml_image_requirements.txt | 42 ++++----- .../py313/base_image_requirements.txt | 94 ++++++++++--------- .../py39/base_image_requirements.txt | 34 +++---- .../container/py39/ml_image_requirements.txt | 38 ++++---- 9 files changed, 200 insertions(+), 196 deletions(-) diff --git a/sdks/python/container/py310/base_image_requirements.txt b/sdks/python/container/py310/base_image_requirements.txt index f60f925b7257..586e6beaaecf 100644 --- a/sdks/python/container/py310/base_image_requirements.txt +++ b/sdks/python/container/py310/base_image_requirements.txt @@ -23,7 +23,7 @@ aiofiles==24.1.0 aiohappyeyeballs==2.6.1 -aiohttp==3.12.14 +aiohttp==3.12.15 aiosignal==1.4.0 annotated-types==0.7.0 anyio==4.9.0 @@ -33,9 +33,9 @@ attrs==25.3.0 backports.tarfile==1.2.0 beautifulsoup4==4.13.4 bs4==0.0.2 -build==1.2.2.post1 +build==1.3.0 cachetools==5.5.2 -certifi==2025.7.14 +certifi==2025.8.3 cffi==1.17.1 charset-normalizer==3.4.2 click==8.2.1 @@ -50,9 +50,9 @@ docopt==0.6.2 docstring_parser==0.17.0 exceptiongroup==1.3.0 execnet==2.1.1 -fastavro==1.11.1 +fastavro==1.12.0 fasteners==0.19 -freezegun==1.5.3 +freezegun==1.5.4 frozenlist==1.7.0 future==1.0.0 google-api-core==2.25.1 @@ -60,7 +60,7 @@ google-api-python-client==2.177.0 google-apitools==0.5.31 google-auth==2.40.3 google-auth-httplib2==0.2.0 -google-cloud-aiplatform==1.105.0 +google-cloud-aiplatform==1.106.0 google-cloud-bigquery==3.35.1 google-cloud-bigquery-storage==2.32.0 google-cloud-bigtable==2.31.0 @@ -69,7 +69,7 @@ google-cloud-datastore==2.21.0 google-cloud-dlp==3.31.0 google-cloud-language==2.17.2 google-cloud-profiler==4.1.0 -google-cloud-pubsub==2.31.0 +google-cloud-pubsub==2.31.1 google-cloud-pubsublite==1.12.0 google-cloud-recommendations-ai==0.10.18 google-cloud-resource-manager==1.14.2 @@ -78,7 +78,7 @@ google-cloud-storage==2.19.0 google-cloud-videointelligence==2.16.2 google-cloud-vision==3.10.2 google-crc32c==1.7.1 -google-genai==1.27.0 +google-genai==1.28.0 google-resumable-media==2.7.2 googleapis-common-protos==1.70.0 greenlet==3.2.3 @@ -92,7 +92,7 @@ hdfs==2.7.3 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -hypothesis==6.136.4 +hypothesis==6.136.7 idna==3.10 importlib_metadata==8.7.0 iniconfig==2.1.0 @@ -109,7 +109,7 @@ keyring==25.6.0 keyrings.google-artifactregistry-auth==1.1.2 MarkupSafe==3.0.2 milvus-lite==2.5.1 -mmh3==5.1.0 +mmh3==5.2.0 mock==5.2.0 more-itertools==10.7.0 multidict==6.6.3 @@ -118,17 +118,17 @@ nltk==3.9.1 numpy==2.2.6 oauth2client==4.1.3 objsize==0.7.1 -opentelemetry-api==1.35.0 -opentelemetry-sdk==1.35.0 -opentelemetry-semantic-conventions==0.56b0 -oracledb==3.2.0 +opentelemetry-api==1.36.0 +opentelemetry-sdk==1.36.0 +opentelemetry-semantic-conventions==0.57b0 +oracledb==3.3.0 orjson==3.11.1 overrides==7.7.0 packaging==25.0 pandas==2.2.3 parameterized==0.9.0 pg8000==1.31.4 -pip==25.1.1 +pip==25.2 pluggy==1.6.0 propcache==0.3.2 proto-plus==1.26.1 @@ -159,7 +159,7 @@ pytz==2025.2 PyYAML==6.0.2 redis==5.3.1 referencing==0.36.2 -regex==2024.11.6 +regex==2025.7.34 requests==2.32.4 requests-mock==1.12.1 rpds-py==0.26.0 @@ -174,7 +174,7 @@ six==1.17.0 sniffio==1.3.1 sortedcontainers==2.4.0 soupsieve==2.7 -SQLAlchemy==2.0.41 +SQLAlchemy==2.0.42 sqlalchemy_pytds==1.0.2 sqlparse==0.5.3 tenacity==8.5.0 diff --git a/sdks/python/container/py310/ml_image_requirements.txt b/sdks/python/container/py310/ml_image_requirements.txt index 52a69c52dff3..2dfb01c87a08 100644 --- a/sdks/python/container/py310/ml_image_requirements.txt +++ b/sdks/python/container/py310/ml_image_requirements.txt @@ -24,7 +24,7 @@ absl-py==2.3.1 aiofiles==24.1.0 aiohappyeyeballs==2.6.1 -aiohttp==3.12.14 +aiohttp==3.12.15 aiosignal==1.4.0 annotated-types==0.7.0 anyio==4.9.0 @@ -35,9 +35,9 @@ attrs==25.3.0 backports.tarfile==1.2.0 beautifulsoup4==4.13.4 bs4==0.0.2 -build==1.2.2.post1 +build==1.3.0 cachetools==5.5.2 -certifi==2025.7.14 +certifi==2025.8.3 cffi==1.17.1 charset-normalizer==3.4.2 click==8.2.1 @@ -52,11 +52,11 @@ docopt==0.6.2 docstring_parser==0.17.0 exceptiongroup==1.3.0 execnet==2.1.1 -fastavro==1.11.1 +fastavro==1.12.0 fasteners==0.19 filelock==3.18.0 flatbuffers==25.2.10 -freezegun==1.5.3 +freezegun==1.5.4 frozenlist==1.7.0 fsspec==2025.7.0 future==1.0.0 @@ -66,7 +66,7 @@ google-api-python-client==2.177.0 google-apitools==0.5.31 google-auth==2.40.3 google-auth-httplib2==0.2.0 -google-cloud-aiplatform==1.105.0 +google-cloud-aiplatform==1.106.0 google-cloud-bigquery==3.35.1 google-cloud-bigquery-storage==2.32.0 google-cloud-bigtable==2.31.0 @@ -75,7 +75,7 @@ google-cloud-datastore==2.21.0 google-cloud-dlp==3.31.0 google-cloud-language==2.17.2 google-cloud-profiler==4.1.0 -google-cloud-pubsub==2.31.0 +google-cloud-pubsub==2.31.1 google-cloud-pubsublite==1.12.0 google-cloud-recommendations-ai==0.10.18 google-cloud-resource-manager==1.14.2 @@ -84,7 +84,7 @@ google-cloud-storage==2.19.0 google-cloud-videointelligence==2.16.2 google-cloud-vision==3.10.2 google-crc32c==1.7.1 -google-genai==1.27.0 +google-genai==1.28.0 google-pasta==0.2.0 google-resumable-media==2.7.2 googleapis-common-protos==1.70.0 @@ -101,8 +101,8 @@ hf-xet==1.1.5 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -huggingface-hub==0.34.1 -hypothesis==6.136.4 +huggingface-hub==0.34.3 +hypothesis==6.136.7 idna==3.10 importlib_metadata==8.7.0 iniconfig==2.1.0 @@ -115,7 +115,7 @@ joblib==1.5.1 jsonpickle==3.4.2 jsonschema==4.25.0 jsonschema-specifications==2025.4.1 -keras==3.10.0 +keras==3.11.1 keyring==25.6.0 keyrings.google-artifactregistry-auth==1.1.2 libclang==18.1.1 @@ -125,7 +125,7 @@ MarkupSafe==3.0.2 mdurl==0.1.2 milvus-lite==2.5.1 ml-dtypes==0.3.2 -mmh3==5.1.0 +mmh3==5.2.0 mock==5.2.0 more-itertools==10.7.0 mpmath==1.3.0 @@ -151,19 +151,19 @@ nvidia-nvjitlink-cu12==12.6.85 nvidia-nvtx-cu12==12.6.77 oauth2client==4.1.3 objsize==0.7.1 -opentelemetry-api==1.35.0 -opentelemetry-sdk==1.35.0 -opentelemetry-semantic-conventions==0.56b0 +opentelemetry-api==1.36.0 +opentelemetry-sdk==1.36.0 +opentelemetry-semantic-conventions==0.57b0 opt_einsum==3.4.0 optree==0.17.0 -oracledb==3.2.0 +oracledb==3.3.0 orjson==3.11.1 overrides==7.7.0 packaging==25.0 pandas==2.2.3 parameterized==0.9.0 pg8000==1.31.4 -pip==25.1.1 +pip==25.2 pluggy==1.6.0 propcache==0.3.2 proto-plus==1.26.1 @@ -195,7 +195,7 @@ pytz==2025.2 PyYAML==6.0.2 redis==5.3.1 referencing==0.36.2 -regex==2024.11.6 +regex==2025.7.34 requests==2.32.4 requests-mock==1.12.1 rich==14.1.0 @@ -212,7 +212,7 @@ six==1.17.0 sniffio==1.3.1 sortedcontainers==2.4.0 soupsieve==2.7 -SQLAlchemy==2.0.41 +SQLAlchemy==2.0.42 sqlalchemy_pytds==1.0.2 sqlparse==0.5.3 sympy==1.14.0 @@ -225,7 +225,7 @@ tensorflow-io-gcs-filesystem==0.37.1 termcolor==3.1.0 testcontainers==4.12.0 threadpoolctl==3.6.0 -tokenizers==0.21.2 +tokenizers==0.21.4 tomli==2.2.1 torch==2.7.1 tqdm==4.67.1 diff --git a/sdks/python/container/py311/base_image_requirements.txt b/sdks/python/container/py311/base_image_requirements.txt index 4bf226933fac..65ae772a3b38 100644 --- a/sdks/python/container/py311/base_image_requirements.txt +++ b/sdks/python/container/py311/base_image_requirements.txt @@ -23,7 +23,7 @@ aiofiles==24.1.0 aiohappyeyeballs==2.6.1 -aiohttp==3.12.14 +aiohttp==3.12.15 aiosignal==1.4.0 annotated-types==0.7.0 anyio==4.9.0 @@ -32,9 +32,9 @@ attrs==25.3.0 backports.tarfile==1.2.0 beautifulsoup4==4.13.4 bs4==0.0.2 -build==1.2.2.post1 +build==1.3.0 cachetools==5.5.2 -certifi==2025.7.14 +certifi==2025.8.3 cffi==1.17.1 charset-normalizer==3.4.2 click==8.2.1 @@ -48,9 +48,9 @@ docker==7.1.0 docopt==0.6.2 docstring_parser==0.17.0 execnet==2.1.1 -fastavro==1.11.1 +fastavro==1.12.0 fasteners==0.19 -freezegun==1.5.3 +freezegun==1.5.4 frozenlist==1.7.0 future==1.0.0 google-api-core==2.25.1 @@ -58,7 +58,7 @@ google-api-python-client==2.177.0 google-apitools==0.5.31 google-auth==2.40.3 google-auth-httplib2==0.2.0 -google-cloud-aiplatform==1.105.0 +google-cloud-aiplatform==1.106.0 google-cloud-bigquery==3.35.1 google-cloud-bigquery-storage==2.32.0 google-cloud-bigtable==2.31.0 @@ -67,7 +67,7 @@ google-cloud-datastore==2.21.0 google-cloud-dlp==3.31.0 google-cloud-language==2.17.2 google-cloud-profiler==4.1.0 -google-cloud-pubsub==2.31.0 +google-cloud-pubsub==2.31.1 google-cloud-pubsublite==1.12.0 google-cloud-recommendations-ai==0.10.18 google-cloud-resource-manager==1.14.2 @@ -76,7 +76,7 @@ google-cloud-storage==2.19.0 google-cloud-videointelligence==2.16.2 google-cloud-vision==3.10.2 google-crc32c==1.7.1 -google-genai==1.27.0 +google-genai==1.28.0 google-resumable-media==2.7.2 googleapis-common-protos==1.70.0 greenlet==3.2.3 @@ -90,7 +90,7 @@ hdfs==2.7.3 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -hypothesis==6.136.4 +hypothesis==6.136.7 idna==3.10 importlib_metadata==8.7.0 iniconfig==2.1.0 @@ -107,7 +107,7 @@ keyring==25.6.0 keyrings.google-artifactregistry-auth==1.1.2 MarkupSafe==3.0.2 milvus-lite==2.5.1 -mmh3==5.1.0 +mmh3==5.2.0 mock==5.2.0 more-itertools==10.7.0 multidict==6.6.3 @@ -116,17 +116,17 @@ nltk==3.9.1 numpy==2.2.6 oauth2client==4.1.3 objsize==0.7.1 -opentelemetry-api==1.35.0 -opentelemetry-sdk==1.35.0 -opentelemetry-semantic-conventions==0.56b0 -oracledb==3.2.0 +opentelemetry-api==1.36.0 +opentelemetry-sdk==1.36.0 +opentelemetry-semantic-conventions==0.57b0 +oracledb==3.3.0 orjson==3.11.1 overrides==7.7.0 packaging==25.0 pandas==2.2.3 parameterized==0.9.0 pg8000==1.31.4 -pip==25.1.1 +pip==25.2 pluggy==1.6.0 propcache==0.3.2 proto-plus==1.26.1 @@ -157,13 +157,13 @@ pytz==2025.2 PyYAML==6.0.2 redis==5.3.1 referencing==0.36.2 -regex==2024.11.6 +regex==2025.7.34 requests==2.32.4 requests-mock==1.12.1 rpds-py==0.26.0 rsa==4.9.1 scikit-learn==1.7.1 -scipy==1.16.0 +scipy==1.16.1 scramp==1.4.6 SecretStorage==3.3.3 setuptools==80.9.0 @@ -172,7 +172,7 @@ six==1.17.0 sniffio==1.3.1 sortedcontainers==2.4.0 soupsieve==2.7 -SQLAlchemy==2.0.41 +SQLAlchemy==2.0.42 sqlalchemy_pytds==1.0.2 sqlparse==0.5.3 tenacity==8.5.0 diff --git a/sdks/python/container/py311/ml_image_requirements.txt b/sdks/python/container/py311/ml_image_requirements.txt index fda06246db70..a6f813c0b159 100644 --- a/sdks/python/container/py311/ml_image_requirements.txt +++ b/sdks/python/container/py311/ml_image_requirements.txt @@ -24,7 +24,7 @@ absl-py==2.3.1 aiofiles==24.1.0 aiohappyeyeballs==2.6.1 -aiohttp==3.12.14 +aiohttp==3.12.15 aiosignal==1.4.0 annotated-types==0.7.0 anyio==4.9.0 @@ -34,9 +34,9 @@ attrs==25.3.0 backports.tarfile==1.2.0 beautifulsoup4==4.13.4 bs4==0.0.2 -build==1.2.2.post1 +build==1.3.0 cachetools==5.5.2 -certifi==2025.7.14 +certifi==2025.8.3 cffi==1.17.1 charset-normalizer==3.4.2 click==8.2.1 @@ -50,11 +50,11 @@ docker==7.1.0 docopt==0.6.2 docstring_parser==0.17.0 execnet==2.1.1 -fastavro==1.11.1 +fastavro==1.12.0 fasteners==0.19 filelock==3.18.0 flatbuffers==25.2.10 -freezegun==1.5.3 +freezegun==1.5.4 frozenlist==1.7.0 fsspec==2025.7.0 future==1.0.0 @@ -64,7 +64,7 @@ google-api-python-client==2.177.0 google-apitools==0.5.31 google-auth==2.40.3 google-auth-httplib2==0.2.0 -google-cloud-aiplatform==1.105.0 +google-cloud-aiplatform==1.106.0 google-cloud-bigquery==3.35.1 google-cloud-bigquery-storage==2.32.0 google-cloud-bigtable==2.31.0 @@ -73,7 +73,7 @@ google-cloud-datastore==2.21.0 google-cloud-dlp==3.31.0 google-cloud-language==2.17.2 google-cloud-profiler==4.1.0 -google-cloud-pubsub==2.31.0 +google-cloud-pubsub==2.31.1 google-cloud-pubsublite==1.12.0 google-cloud-recommendations-ai==0.10.18 google-cloud-resource-manager==1.14.2 @@ -82,7 +82,7 @@ google-cloud-storage==2.19.0 google-cloud-videointelligence==2.16.2 google-cloud-vision==3.10.2 google-crc32c==1.7.1 -google-genai==1.27.0 +google-genai==1.28.0 google-pasta==0.2.0 google-resumable-media==2.7.2 googleapis-common-protos==1.70.0 @@ -99,8 +99,8 @@ hf-xet==1.1.5 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -huggingface-hub==0.34.1 -hypothesis==6.136.4 +huggingface-hub==0.34.3 +hypothesis==6.136.7 idna==3.10 importlib_metadata==8.7.0 iniconfig==2.1.0 @@ -113,7 +113,7 @@ joblib==1.5.1 jsonpickle==3.4.2 jsonschema==4.25.0 jsonschema-specifications==2025.4.1 -keras==3.10.0 +keras==3.11.1 keyring==25.6.0 keyrings.google-artifactregistry-auth==1.1.2 libclang==18.1.1 @@ -123,7 +123,7 @@ MarkupSafe==3.0.2 mdurl==0.1.2 milvus-lite==2.5.1 ml-dtypes==0.3.2 -mmh3==5.1.0 +mmh3==5.2.0 mock==5.2.0 more-itertools==10.7.0 mpmath==1.3.0 @@ -149,19 +149,19 @@ nvidia-nvjitlink-cu12==12.6.85 nvidia-nvtx-cu12==12.6.77 oauth2client==4.1.3 objsize==0.7.1 -opentelemetry-api==1.35.0 -opentelemetry-sdk==1.35.0 -opentelemetry-semantic-conventions==0.56b0 +opentelemetry-api==1.36.0 +opentelemetry-sdk==1.36.0 +opentelemetry-semantic-conventions==0.57b0 opt_einsum==3.4.0 optree==0.17.0 -oracledb==3.2.0 +oracledb==3.3.0 orjson==3.11.1 overrides==7.7.0 packaging==25.0 pandas==2.2.3 parameterized==0.9.0 pg8000==1.31.4 -pip==25.1.1 +pip==25.2 pluggy==1.6.0 propcache==0.3.2 proto-plus==1.26.1 @@ -193,7 +193,7 @@ pytz==2025.2 PyYAML==6.0.2 redis==5.3.1 referencing==0.36.2 -regex==2024.11.6 +regex==2025.7.34 requests==2.32.4 requests-mock==1.12.1 rich==14.1.0 @@ -201,7 +201,7 @@ rpds-py==0.26.0 rsa==4.9.1 safetensors==0.5.3 scikit-learn==1.7.1 -scipy==1.16.0 +scipy==1.16.1 scramp==1.4.6 SecretStorage==3.3.3 setuptools==80.9.0 @@ -210,7 +210,7 @@ six==1.17.0 sniffio==1.3.1 sortedcontainers==2.4.0 soupsieve==2.7 -SQLAlchemy==2.0.41 +SQLAlchemy==2.0.42 sqlalchemy_pytds==1.0.2 sqlparse==0.5.3 sympy==1.14.0 @@ -223,7 +223,7 @@ tensorflow-io-gcs-filesystem==0.37.1 termcolor==3.1.0 testcontainers==4.12.0 threadpoolctl==3.6.0 -tokenizers==0.21.2 +tokenizers==0.21.4 torch==2.7.1 tqdm==4.67.1 transformers==4.48.3 diff --git a/sdks/python/container/py312/base_image_requirements.txt b/sdks/python/container/py312/base_image_requirements.txt index bbbf56883262..621cf1ecbe12 100644 --- a/sdks/python/container/py312/base_image_requirements.txt +++ b/sdks/python/container/py312/base_image_requirements.txt @@ -23,7 +23,7 @@ aiofiles==24.1.0 aiohappyeyeballs==2.6.1 -aiohttp==3.12.14 +aiohttp==3.12.15 aiosignal==1.4.0 annotated-types==0.7.0 anyio==4.9.0 @@ -31,9 +31,9 @@ asn1crypto==1.5.1 attrs==25.3.0 beautifulsoup4==4.13.4 bs4==0.0.2 -build==1.2.2.post1 +build==1.3.0 cachetools==5.5.2 -certifi==2025.7.14 +certifi==2025.8.3 cffi==1.17.1 charset-normalizer==3.4.2 click==8.2.1 @@ -47,9 +47,9 @@ docker==7.1.0 docopt==0.6.2 docstring_parser==0.17.0 execnet==2.1.1 -fastavro==1.11.1 +fastavro==1.12.0 fasteners==0.19 -freezegun==1.5.3 +freezegun==1.5.4 frozenlist==1.7.0 future==1.0.0 google-api-core==2.25.1 @@ -57,7 +57,7 @@ google-api-python-client==2.177.0 google-apitools==0.5.31 google-auth==2.40.3 google-auth-httplib2==0.2.0 -google-cloud-aiplatform==1.105.0 +google-cloud-aiplatform==1.106.0 google-cloud-bigquery==3.35.1 google-cloud-bigquery-storage==2.32.0 google-cloud-bigtable==2.31.0 @@ -66,7 +66,7 @@ google-cloud-datastore==2.21.0 google-cloud-dlp==3.31.0 google-cloud-language==2.17.2 google-cloud-profiler==4.1.0 -google-cloud-pubsub==2.31.0 +google-cloud-pubsub==2.31.1 google-cloud-pubsublite==1.12.0 google-cloud-recommendations-ai==0.10.18 google-cloud-resource-manager==1.14.2 @@ -75,7 +75,7 @@ google-cloud-storage==2.19.0 google-cloud-videointelligence==2.16.2 google-cloud-vision==3.10.2 google-crc32c==1.7.1 -google-genai==1.27.0 +google-genai==1.28.0 google-resumable-media==2.7.2 googleapis-common-protos==1.70.0 greenlet==3.2.3 @@ -89,7 +89,7 @@ hdfs==2.7.3 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -hypothesis==6.136.4 +hypothesis==6.136.7 idna==3.10 importlib_metadata==8.7.0 iniconfig==2.1.0 @@ -106,7 +106,7 @@ keyring==25.6.0 keyrings.google-artifactregistry-auth==1.1.2 MarkupSafe==3.0.2 milvus-lite==2.5.1 -mmh3==5.1.0 +mmh3==5.2.0 mock==5.2.0 more-itertools==10.7.0 multidict==6.6.3 @@ -115,17 +115,17 @@ nltk==3.9.1 numpy==2.2.6 oauth2client==4.1.3 objsize==0.7.1 -opentelemetry-api==1.35.0 -opentelemetry-sdk==1.35.0 -opentelemetry-semantic-conventions==0.56b0 -oracledb==3.2.0 +opentelemetry-api==1.36.0 +opentelemetry-sdk==1.36.0 +opentelemetry-semantic-conventions==0.57b0 +oracledb==3.3.0 orjson==3.11.1 overrides==7.7.0 packaging==25.0 pandas==2.2.3 parameterized==0.9.0 pg8000==1.31.4 -pip==25.1.1 +pip==25.2 pluggy==1.6.0 propcache==0.3.2 proto-plus==1.26.1 @@ -156,13 +156,13 @@ pytz==2025.2 PyYAML==6.0.2 redis==5.3.1 referencing==0.36.2 -regex==2024.11.6 +regex==2025.7.34 requests==2.32.4 requests-mock==1.12.1 rpds-py==0.26.0 rsa==4.9.1 scikit-learn==1.7.1 -scipy==1.16.0 +scipy==1.16.1 scramp==1.4.6 SecretStorage==3.3.3 setuptools==80.9.0 @@ -171,7 +171,7 @@ six==1.17.0 sniffio==1.3.1 sortedcontainers==2.4.0 soupsieve==2.7 -SQLAlchemy==2.0.41 +SQLAlchemy==2.0.42 sqlalchemy_pytds==1.0.2 sqlparse==0.5.3 tenacity==8.5.0 diff --git a/sdks/python/container/py312/ml_image_requirements.txt b/sdks/python/container/py312/ml_image_requirements.txt index c737cef6f8b5..a7e5b27c9817 100644 --- a/sdks/python/container/py312/ml_image_requirements.txt +++ b/sdks/python/container/py312/ml_image_requirements.txt @@ -24,7 +24,7 @@ absl-py==2.3.1 aiofiles==24.1.0 aiohappyeyeballs==2.6.1 -aiohttp==3.12.14 +aiohttp==3.12.15 aiosignal==1.4.0 annotated-types==0.7.0 anyio==4.9.0 @@ -33,9 +33,9 @@ astunparse==1.6.3 attrs==25.3.0 beautifulsoup4==4.13.4 bs4==0.0.2 -build==1.2.2.post1 +build==1.3.0 cachetools==5.5.2 -certifi==2025.7.14 +certifi==2025.8.3 cffi==1.17.1 charset-normalizer==3.4.2 click==8.2.1 @@ -49,11 +49,11 @@ docker==7.1.0 docopt==0.6.2 docstring_parser==0.17.0 execnet==2.1.1 -fastavro==1.11.1 +fastavro==1.12.0 fasteners==0.19 filelock==3.18.0 flatbuffers==25.2.10 -freezegun==1.5.3 +freezegun==1.5.4 frozenlist==1.7.0 fsspec==2025.7.0 future==1.0.0 @@ -63,7 +63,7 @@ google-api-python-client==2.177.0 google-apitools==0.5.31 google-auth==2.40.3 google-auth-httplib2==0.2.0 -google-cloud-aiplatform==1.105.0 +google-cloud-aiplatform==1.106.0 google-cloud-bigquery==3.35.1 google-cloud-bigquery-storage==2.32.0 google-cloud-bigtable==2.31.0 @@ -72,7 +72,7 @@ google-cloud-datastore==2.21.0 google-cloud-dlp==3.31.0 google-cloud-language==2.17.2 google-cloud-profiler==4.1.0 -google-cloud-pubsub==2.31.0 +google-cloud-pubsub==2.31.1 google-cloud-pubsublite==1.12.0 google-cloud-recommendations-ai==0.10.18 google-cloud-resource-manager==1.14.2 @@ -81,7 +81,7 @@ google-cloud-storage==2.19.0 google-cloud-videointelligence==2.16.2 google-cloud-vision==3.10.2 google-crc32c==1.7.1 -google-genai==1.27.0 +google-genai==1.28.0 google-pasta==0.2.0 google-resumable-media==2.7.2 googleapis-common-protos==1.70.0 @@ -98,8 +98,8 @@ hf-xet==1.1.5 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -huggingface-hub==0.34.1 -hypothesis==6.136.4 +huggingface-hub==0.34.3 +hypothesis==6.136.7 idna==3.10 importlib_metadata==8.7.0 iniconfig==2.1.0 @@ -112,7 +112,7 @@ joblib==1.5.1 jsonpickle==3.4.2 jsonschema==4.25.0 jsonschema-specifications==2025.4.1 -keras==3.10.0 +keras==3.11.1 keyring==25.6.0 keyrings.google-artifactregistry-auth==1.1.2 libclang==18.1.1 @@ -122,7 +122,7 @@ MarkupSafe==3.0.2 mdurl==0.1.2 milvus-lite==2.5.1 ml-dtypes==0.3.2 -mmh3==5.1.0 +mmh3==5.2.0 mock==5.2.0 more-itertools==10.7.0 mpmath==1.3.0 @@ -148,19 +148,19 @@ nvidia-nvjitlink-cu12==12.6.85 nvidia-nvtx-cu12==12.6.77 oauth2client==4.1.3 objsize==0.7.1 -opentelemetry-api==1.35.0 -opentelemetry-sdk==1.35.0 -opentelemetry-semantic-conventions==0.56b0 +opentelemetry-api==1.36.0 +opentelemetry-sdk==1.36.0 +opentelemetry-semantic-conventions==0.57b0 opt_einsum==3.4.0 optree==0.17.0 -oracledb==3.2.0 +oracledb==3.3.0 orjson==3.11.1 overrides==7.7.0 packaging==25.0 pandas==2.2.3 parameterized==0.9.0 pg8000==1.31.4 -pip==25.1.1 +pip==25.2 pluggy==1.6.0 propcache==0.3.2 proto-plus==1.26.1 @@ -192,7 +192,7 @@ pytz==2025.2 PyYAML==6.0.2 redis==5.3.1 referencing==0.36.2 -regex==2024.11.6 +regex==2025.7.34 requests==2.32.4 requests-mock==1.12.1 rich==14.1.0 @@ -200,7 +200,7 @@ rpds-py==0.26.0 rsa==4.9.1 safetensors==0.5.3 scikit-learn==1.7.1 -scipy==1.16.0 +scipy==1.16.1 scramp==1.4.6 SecretStorage==3.3.3 setuptools==80.9.0 @@ -209,7 +209,7 @@ six==1.17.0 sniffio==1.3.1 sortedcontainers==2.4.0 soupsieve==2.7 -SQLAlchemy==2.0.41 +SQLAlchemy==2.0.42 sqlalchemy_pytds==1.0.2 sqlparse==0.5.3 sympy==1.14.0 @@ -221,7 +221,7 @@ tensorflow-cpu-aws==2.16.2;platform_machine=="aarch64" termcolor==3.1.0 testcontainers==4.12.0 threadpoolctl==3.6.0 -tokenizers==0.21.2 +tokenizers==0.21.4 torch==2.7.1 tqdm==4.67.1 transformers==4.48.3 diff --git a/sdks/python/container/py313/base_image_requirements.txt b/sdks/python/container/py313/base_image_requirements.txt index ea856d786160..c35b33c2d940 100644 --- a/sdks/python/container/py313/base_image_requirements.txt +++ b/sdks/python/container/py313/base_image_requirements.txt @@ -23,106 +23,107 @@ aiofiles==24.1.0 aiohappyeyeballs==2.6.1 -aiohttp==3.12.12 -aiosignal==1.3.2 +aiohttp==3.12.15 +aiosignal==1.4.0 annotated-types==0.7.0 anyio==4.9.0 asn1crypto==1.5.1 attrs==25.3.0 beautifulsoup4==4.13.4 bs4==0.0.2 -build==1.2.2.post1 +build==1.3.0 cachetools==5.5.2 -certifi==2025.4.26 +certifi==2025.8.3 cffi==1.17.1 charset-normalizer==3.4.2 click==8.2.1 -cloud-sql-python-connector==1.18.2 +cloud-sql-python-connector==1.18.3 crcmod==1.7 -cryptography==45.0.4 +cryptography==45.0.5 Cython==3.1.2 -deprecation==2.1.0 dill==0.3.1.1 dnspython==2.7.0 docker==7.1.0 docopt==0.6.2 -docstring_parser==0.16 +docstring_parser==0.17.0 execnet==2.1.1 -fastavro==1.11.1 +fastavro==1.12.0 fasteners==0.19 -freezegun==1.5.2 +freezegun==1.5.4 frozenlist==1.7.0 future==1.0.0 -google-api-core==2.25.0 +google-api-core==2.25.1 google-apitools==0.5.32 google-auth==2.40.3 google-auth-httplib2==0.2.0 -google-cloud-aiplatform==1.97.0 -google-cloud-bigquery==3.34.0 +google-cloud-aiplatform==1.106.0 +google-cloud-bigquery==3.35.1 google-cloud-bigquery-storage==2.32.0 google-cloud-bigtable==2.31.0 google-cloud-core==2.4.3 google-cloud-datastore==2.21.0 -google-cloud-dlp==3.30.0 +google-cloud-dlp==3.31.0 google-cloud-language==2.17.2 -google-cloud-pubsub==2.30.0 +google-cloud-pubsub==2.31.1 google-cloud-pubsublite==1.12.0 google-cloud-recommendations-ai==0.10.18 google-cloud-resource-manager==1.14.2 -google-cloud-spanner==3.55.0 +google-cloud-spanner==3.56.0 google-cloud-storage==2.19.0 google-cloud-videointelligence==2.16.2 google-cloud-vision==3.10.2 google-crc32c==1.7.1 -google-genai==1.20.0 +google-genai==1.28.0 google-resumable-media==2.7.2 googleapis-common-protos==1.70.0 greenlet==3.2.3 grpc-google-iam-v1==0.14.2 grpc-interceptor==0.15.4 -grpcio==1.73.0 -grpcio-status==1.71.0 +grpcio==1.67.1 +grpcio-status==1.67.1 guppy3==3.1.5 h11==0.16.0 hdfs==2.7.3 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -hypothesis==6.135.7 +hypothesis==6.136.7 idna==3.10 importlib_metadata==8.7.0 iniconfig==2.1.0 jaraco.classes==3.4.0 jaraco.context==6.0.1 -jaraco.functools==4.1.0 +jaraco.functools==4.2.1 jeepney==0.9.0 Jinja2==3.1.6 joblib==1.5.1 jsonpickle==3.4.2 -jsonschema==4.24.0 +jsonschema==4.25.0 jsonschema-specifications==2025.4.1 -kafka-python==2.2.11 keyring==25.6.0 keyrings.google-artifactregistry-auth==1.1.2 MarkupSafe==3.0.2 -mmh3==5.1.0 +milvus-lite==2.5.1 +mmh3==5.2.0 mock==5.2.0 more-itertools==10.7.0 -multidict==6.4.4 -mysql-connector-python==9.3.0 +multidict==6.6.3 +mysql-connector-python==9.4.0 nltk==3.9.1 numpy==2.2.6 oauth2client==4.1.3 objsize==0.7.1 -opentelemetry-api==1.34.1 -opentelemetry-sdk==1.34.1 -opentelemetry-semantic-conventions==0.55b1 -orjson==3.10.18 +opentelemetry-api==1.36.0 +opentelemetry-sdk==1.36.0 +opentelemetry-semantic-conventions==0.57b0 +oracledb==3.3.0 +orjson==3.11.1 overrides==7.7.0 packaging==25.0 pandas==2.2.3 parameterized==0.9.0 -pg8000==1.31.2 +pg8000==1.31.4 +pip==25.2 pluggy==1.6.0 propcache==0.3.2 proto-plus==1.26.1 @@ -133,32 +134,34 @@ pyarrow-hotfix==0.7 pyasn1==0.6.1 pyasn1_modules==0.4.2 pycparser==2.22 -pydantic==2.11.5 +pydantic==2.11.7 pydantic_core==2.33.2 pydot==1.4.2 PyHamcrest==2.1.0 -PyJWT==2.9.0 -pymongo==4.13.1 +PyJWT==2.10.1 +pymilvus==2.5.14 +pymongo==4.13.2 PyMySQL==1.1.1 pyparsing==3.2.3 pyproject_hooks==1.2.0 pytest==7.4.4 pytest-timeout==2.4.0 -pytest-xdist==3.7.0 +pytest-xdist==3.8.0 python-dateutil==2.9.0.post0 +python-dotenv==1.1.1 python-tds==1.16.1 pytz==2025.2 PyYAML==6.0.2 -redis==5.3.0 +redis==5.3.1 referencing==0.36.2 -regex==2024.11.6 +regex==2025.7.34 requests==2.32.4 requests-mock==1.12.1 -rpds-py==0.25.1 +rpds-py==0.26.0 rsa==4.9.1 -scikit-learn==1.7.0 -scipy==1.15.3 -scramp==1.4.5 +scikit-learn==1.7.1 +scipy==1.16.1 +scramp==1.4.6 SecretStorage==3.3.3 setuptools==80.9.0 shapely==2.1.1 @@ -166,17 +169,18 @@ six==1.17.0 sniffio==1.3.1 sortedcontainers==2.4.0 soupsieve==2.7 -SQLAlchemy==2.0.41 +SQLAlchemy==2.0.42 sqlalchemy_pytds==1.0.2 sqlparse==0.5.3 tenacity==8.5.0 -testcontainers==3.7.1 +testcontainers==4.12.0 threadpoolctl==3.6.0 tqdm==4.67.1 typing-inspection==0.4.1 -typing_extensions==4.14.0 +typing_extensions==4.14.1 tzdata==2025.2 -urllib3==2.4.0 +ujson==5.10.0 +urllib3==2.5.0 virtualenv-clone==0.5.7 websockets==15.0.1 wheel==0.45.1 diff --git a/sdks/python/container/py39/base_image_requirements.txt b/sdks/python/container/py39/base_image_requirements.txt index 2cc3da6cfc17..57ab9ba11edc 100644 --- a/sdks/python/container/py39/base_image_requirements.txt +++ b/sdks/python/container/py39/base_image_requirements.txt @@ -23,7 +23,7 @@ aiofiles==24.1.0 aiohappyeyeballs==2.6.1 -aiohttp==3.12.14 +aiohttp==3.12.15 aiosignal==1.4.0 annotated-types==0.7.0 anyio==4.9.0 @@ -33,9 +33,9 @@ attrs==25.3.0 backports.tarfile==1.2.0 beautifulsoup4==4.13.4 bs4==0.0.2 -build==1.2.2.post1 +build==1.3.0 cachetools==5.5.2 -certifi==2025.7.14 +certifi==2025.8.3 cffi==1.17.1 charset-normalizer==3.4.2 click==8.1.8 @@ -50,9 +50,9 @@ docopt==0.6.2 docstring_parser==0.17.0 exceptiongroup==1.3.0 execnet==2.1.1 -fastavro==1.11.1 +fastavro==1.12.0 fasteners==0.19 -freezegun==1.5.3 +freezegun==1.5.4 frozenlist==1.7.0 future==1.0.0 google-api-core==2.25.1 @@ -60,7 +60,7 @@ google-api-python-client==2.177.0 google-apitools==0.5.31 google-auth==2.40.3 google-auth-httplib2==0.2.0 -google-cloud-aiplatform==1.105.0 +google-cloud-aiplatform==1.106.0 google-cloud-bigquery==3.35.1 google-cloud-bigquery-storage==2.32.0 google-cloud-bigtable==2.31.0 @@ -69,7 +69,7 @@ google-cloud-datastore==2.21.0 google-cloud-dlp==3.31.0 google-cloud-language==2.17.2 google-cloud-profiler==4.1.0 -google-cloud-pubsub==2.31.0 +google-cloud-pubsub==2.31.1 google-cloud-pubsublite==1.12.0 google-cloud-recommendations-ai==0.10.18 google-cloud-resource-manager==1.14.2 @@ -78,7 +78,7 @@ google-cloud-storage==2.19.0 google-cloud-videointelligence==2.16.2 google-cloud-vision==3.10.2 google-crc32c==1.7.1 -google-genai==1.27.0 +google-genai==1.28.0 google-resumable-media==2.7.2 googleapis-common-protos==1.70.0 greenlet==3.2.3 @@ -92,7 +92,7 @@ hdfs==2.7.3 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -hypothesis==6.136.4 +hypothesis==6.136.7 idna==3.10 importlib_metadata==8.7.0 iniconfig==2.1.0 @@ -109,7 +109,7 @@ keyring==25.6.0 keyrings.google-artifactregistry-auth==1.1.2 MarkupSafe==3.0.2 milvus-lite==2.5.1 -mmh3==5.1.0 +mmh3==5.2.0 mock==5.2.0 more-itertools==10.7.0 multidict==6.6.3 @@ -118,17 +118,17 @@ nltk==3.9.1 numpy==2.0.2 oauth2client==4.1.3 objsize==0.7.1 -opentelemetry-api==1.35.0 -opentelemetry-sdk==1.35.0 -opentelemetry-semantic-conventions==0.56b0 -oracledb==3.2.0 +opentelemetry-api==1.36.0 +opentelemetry-sdk==1.36.0 +opentelemetry-semantic-conventions==0.57b0 +oracledb==3.3.0 orjson==3.11.1 overrides==7.7.0 packaging==25.0 pandas==2.2.3 parameterized==0.9.0 pg8000==1.31.4 -pip==25.1.1 +pip==25.2 pluggy==1.6.0 propcache==0.3.2 proto-plus==1.26.1 @@ -159,7 +159,7 @@ pytz==2025.2 PyYAML==6.0.2 redis==5.3.1 referencing==0.36.2 -regex==2024.11.6 +regex==2025.7.34 requests==2.32.4 requests-mock==1.12.1 rpds-py==0.26.0 @@ -174,7 +174,7 @@ six==1.17.0 sniffio==1.3.1 sortedcontainers==2.4.0 soupsieve==2.7 -SQLAlchemy==2.0.41 +SQLAlchemy==2.0.42 sqlalchemy_pytds==1.0.2 sqlparse==0.5.3 tenacity==8.5.0 diff --git a/sdks/python/container/py39/ml_image_requirements.txt b/sdks/python/container/py39/ml_image_requirements.txt index 4b2a30ba1425..064e85acf04e 100644 --- a/sdks/python/container/py39/ml_image_requirements.txt +++ b/sdks/python/container/py39/ml_image_requirements.txt @@ -24,7 +24,7 @@ absl-py==2.3.1 aiofiles==24.1.0 aiohappyeyeballs==2.6.1 -aiohttp==3.12.14 +aiohttp==3.12.15 aiosignal==1.4.0 annotated-types==0.7.0 anyio==4.9.0 @@ -35,9 +35,9 @@ attrs==25.3.0 backports.tarfile==1.2.0 beautifulsoup4==4.13.4 bs4==0.0.2 -build==1.2.2.post1 +build==1.3.0 cachetools==5.5.2 -certifi==2025.7.14 +certifi==2025.8.3 cffi==1.17.1 charset-normalizer==3.4.2 click==8.1.8 @@ -52,11 +52,11 @@ docopt==0.6.2 docstring_parser==0.17.0 exceptiongroup==1.3.0 execnet==2.1.1 -fastavro==1.11.1 +fastavro==1.12.0 fasteners==0.19 filelock==3.18.0 flatbuffers==25.2.10 -freezegun==1.5.3 +freezegun==1.5.4 frozenlist==1.7.0 fsspec==2025.7.0 future==1.0.0 @@ -66,7 +66,7 @@ google-api-python-client==2.177.0 google-apitools==0.5.31 google-auth==2.40.3 google-auth-httplib2==0.2.0 -google-cloud-aiplatform==1.105.0 +google-cloud-aiplatform==1.106.0 google-cloud-bigquery==3.35.1 google-cloud-bigquery-storage==2.32.0 google-cloud-bigtable==2.31.0 @@ -75,7 +75,7 @@ google-cloud-datastore==2.21.0 google-cloud-dlp==3.31.0 google-cloud-language==2.17.2 google-cloud-profiler==4.1.0 -google-cloud-pubsub==2.31.0 +google-cloud-pubsub==2.31.1 google-cloud-pubsublite==1.12.0 google-cloud-recommendations-ai==0.10.18 google-cloud-resource-manager==1.14.2 @@ -84,7 +84,7 @@ google-cloud-storage==2.19.0 google-cloud-videointelligence==2.16.2 google-cloud-vision==3.10.2 google-crc32c==1.7.1 -google-genai==1.27.0 +google-genai==1.28.0 google-pasta==0.2.0 google-resumable-media==2.7.2 googleapis-common-protos==1.70.0 @@ -101,8 +101,8 @@ hf-xet==1.1.5 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -huggingface-hub==0.34.1 -hypothesis==6.136.4 +huggingface-hub==0.34.3 +hypothesis==6.136.7 idna==3.10 importlib_metadata==8.7.0 iniconfig==2.1.0 @@ -125,7 +125,7 @@ MarkupSafe==3.0.2 mdurl==0.1.2 milvus-lite==2.5.1 ml-dtypes==0.3.2 -mmh3==5.1.0 +mmh3==5.2.0 mock==5.2.0 more-itertools==10.7.0 mpmath==1.3.0 @@ -151,19 +151,19 @@ nvidia-nvjitlink-cu12==12.6.85 nvidia-nvtx-cu12==12.6.77 oauth2client==4.1.3 objsize==0.7.1 -opentelemetry-api==1.35.0 -opentelemetry-sdk==1.35.0 -opentelemetry-semantic-conventions==0.56b0 +opentelemetry-api==1.36.0 +opentelemetry-sdk==1.36.0 +opentelemetry-semantic-conventions==0.57b0 opt_einsum==3.4.0 optree==0.17.0 -oracledb==3.2.0 +oracledb==3.3.0 orjson==3.11.1 overrides==7.7.0 packaging==25.0 pandas==2.2.3 parameterized==0.9.0 pg8000==1.31.4 -pip==25.1.1 +pip==25.2 pluggy==1.6.0 propcache==0.3.2 proto-plus==1.26.1 @@ -195,7 +195,7 @@ pytz==2025.2 PyYAML==6.0.2 redis==5.3.1 referencing==0.36.2 -regex==2024.11.6 +regex==2025.7.34 requests==2.32.4 requests-mock==1.12.1 rich==14.1.0 @@ -212,7 +212,7 @@ six==1.17.0 sniffio==1.3.1 sortedcontainers==2.4.0 soupsieve==2.7 -SQLAlchemy==2.0.41 +SQLAlchemy==2.0.42 sqlalchemy_pytds==1.0.2 sqlparse==0.5.3 sympy==1.14.0 @@ -225,7 +225,7 @@ tensorflow-io-gcs-filesystem==0.37.1 termcolor==3.1.0 testcontainers==4.12.0 threadpoolctl==3.6.0 -tokenizers==0.21.2 +tokenizers==0.21.4 tomli==2.2.1 torch==2.7.1 tqdm==4.67.1 From a81fe240e77951f900de3a94b2bbc7fd31caa5fb Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 3 Aug 2025 23:53:05 +0000 Subject: [PATCH 80/83] .github: trigger postcommit python --- .github/trigger_files/beam_PostCommit_Python.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/trigger_files/beam_PostCommit_Python.json b/.github/trigger_files/beam_PostCommit_Python.json index 4aa5f70b72ee..60fcd70d3c3e 100644 --- a/.github/trigger_files/beam_PostCommit_Python.json +++ b/.github/trigger_files/beam_PostCommit_Python.json @@ -1,5 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run.", - "modification": 101 + "modification": 201 } From 3f16f1368c39b2e471c707862ad12d283ca5e8b1 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Fri, 15 Aug 2025 22:22:21 +0000 Subject: [PATCH 81/83] sdks/python: try to fix python doc linting issues --- .../apache_beam/transforms/enrichment_handlers/cloudsql.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py index 3e06b112a746..a5725a945cbd 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py @@ -260,7 +260,8 @@ def __init__( **kwargs, ): """ - Example Usage: + Example usage:: + connection_config = CloudSQLConnectionConfig( db_adapter=DatabaseTypeAdapter.POSTGRESQL, instance_connection_uri="apache-beam-testing:us-central1:itests", From ac0c137eabf175be317b0d245a931501f6be9c4e Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sat, 16 Aug 2025 09:45:48 +0000 Subject: [PATCH 82/83] sdks/python: fix linting issues --- .../apache_beam/transforms/enrichment_handlers/cloudsql.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py index a5725a945cbd..f070158d1c54 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py @@ -281,9 +281,9 @@ def __init__( `ExternalSQLDBConnectionConfig`. This determines how the handler connects to the target SQL database. query_config: Configuration for database queries. Must be one of: - * CustomQueryConfig: For providing a custom query function - * TableFieldsQueryConfig: specifies table, where clause, and field names - * TableFunctionQueryConfig: specifies table, where clause, and val func + * CustomQueryConfig: For providing a custom query function + * TableFieldsQueryConfig: specifies table, where clause, and field names + * TableFunctionQueryConfig: specifies table, where clause, and val func column_names (Optional[list[str]]): List of column names to select from the Cloud SQL table. If not provided, all columns (`*`) are selected. min_batch_size (int): Minimum number of rows to batch together when From 6a668d399b881dfc97d4e234976c736a0ae20563 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Wed, 20 Aug 2025 09:21:26 +0000 Subject: [PATCH 83/83] .github: trigger postcommit python --- .github/trigger_files/beam_PostCommit_Python.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/trigger_files/beam_PostCommit_Python.json b/.github/trigger_files/beam_PostCommit_Python.json index e4f92b51216e..06bd728be6d7 100644 --- a/.github/trigger_files/beam_PostCommit_Python.json +++ b/.github/trigger_files/beam_PostCommit_Python.json @@ -1,5 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run.", - "modification": 389 + "modification": 33 }