diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/bigquery.py b/sdks/python/apache_beam/transforms/enrichment_handlers/bigquery.py index 24a60864c0ca..ea98fb6b0bbd 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/bigquery.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/bigquery.py @@ -202,8 +202,7 @@ def __call__(self, request: Union[beam.Row, List[beam.Row]], *args, **kwargs): "Make sure the values passed in `fields` are the " "keys in the input `beam.Row`." + str(e)) values.extend(current_values) - requests_map.update( - (self.create_row_key(req), req) for val in current_values) + requests_map[self.create_row_key(req)] = req query = raw_query.format(*values) responses_dict = self._execute_query(query) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/bigquery_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/bigquery_it_test.py index da01f06cbc28..dd99e386555e 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/bigquery_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/bigquery_it_test.py @@ -16,10 +16,9 @@ # import functools import logging -import unittest import secrets import time - +import unittest from unittest.mock import MagicMock import pytest @@ -55,6 +54,7 @@ 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 BigQueryEnrichmentIT(unittest.TestCase): bigquery_dataset_id = 'python_enrichment_transform_read_table_' project = "apache-beam-testing"