From 09e61dcdea80ba430ad03ee30bf02d165faeb231 Mon Sep 17 00:00:00 2001 From: ehab20011 Date: Mon, 1 Dec 2025 01:15:28 -0500 Subject: [PATCH 1/2] Fix BigQuery Client not having an explicit ProjectID --- .../apache_beam/io/gcp/bigquery_tools.py | 17 +++++++++---- .../apache_beam/io/gcp/bigquery_tools_test.py | 24 +++++++++++++++++++ 2 files changed, 37 insertions(+), 4 deletions(-) diff --git a/sdks/python/apache_beam/io/gcp/bigquery_tools.py b/sdks/python/apache_beam/io/gcp/bigquery_tools.py index ddab941f9278..f3b87cdfd9f1 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery_tools.py +++ b/sdks/python/apache_beam/io/gcp/bigquery_tools.py @@ -60,6 +60,7 @@ from apache_beam.metrics import monitoring_infos from apache_beam.metrics.metric import Metrics from apache_beam.options import value_provider +from apache_beam.options.pipeline_options import GoogleCloudOptions from apache_beam.options.pipeline_options import PipelineOptions from apache_beam.transforms import DoFn from apache_beam.typehints.row_type import RowTypeConstraint @@ -360,10 +361,18 @@ class BigQueryWrapper(object): HISTOGRAM_METRIC_LOGGER = MetricLogger() def __init__(self, client=None, temp_dataset_id=None, temp_table_ref=None): - self.client = client or BigQueryWrapper._bigquery_client(PipelineOptions()) - self.gcp_bq_client = client or gcp_bigquery.Client( - client_info=ClientInfo( - user_agent="apache-beam-%s" % apache_beam.__version__)) + if client is not None: + self.client = client + self.gcp_bq_client = client + else: + pipeline_options = PipelineOptions() + self.client = BigQueryWrapper._bigquery_client(pipeline_options) + + project = pipeline_options.view_as(GoogleCloudOptions).project + self.gcp_bq_client = gcp_bigquery.Client( + project=project, + client_info=ClientInfo( + user_agent="apache-beam-%s" % apache_beam.__version__)) self._unique_row_id = 0 # For testing scenarios where we pass in a client we do not want a diff --git a/sdks/python/apache_beam/io/gcp/bigquery_tools_test.py b/sdks/python/apache_beam/io/gcp/bigquery_tools_test.py index 2594e6728e0e..067a0d1e2246 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery_tools_test.py +++ b/sdks/python/apache_beam/io/gcp/bigquery_tools_test.py @@ -247,6 +247,30 @@ def test_user_agent_insert_all( call = http_mock.request.mock_calls[-2] self.assertIn('apache-beam-', call[2]['headers']['User-Agent']) + def test_wrapper_uses_injected_client_for_both_low_and_high_level(self): + """If a client is provided, it should be used for both clients.""" + client = mock.Mock() + wrapper = beam.io.gcp.bigquery_tools.BigQueryWrapper(client) + + # Both low-level and high-level clients should be exactly the injected one. + self.assertIs(wrapper.client, client) + self.assertIs(wrapper.gcp_bq_client, client) + + @unittest.skipIf( + beam.io.gcp.bigquery_tools.gcp_bigquery is None, + "bigquery library not available in this env") + @mock.patch('apache_beam.io.gcp.bigquery_tools.gcp_bigquery.Client') + def test_gcp_bigquery_client_is_constructed_with_project_kwarg( + self, mock_bq_client_ctor): + """BigQueryWrapper should always pass an explicit project kwarg.""" + # Constructing the wrapper without an injected client should create a + # google.cloud.bigquery.Client with an explicit "project" keyword arg, + # even if its value is None. + beam.io.gcp.bigquery_tools.BigQueryWrapper() + + _, kwargs = mock_bq_client_ctor.call_args + self.assertIn('project', kwargs) + # the function create_temporary_dataset() in the wrapper does not call # google.cloud.bigquery, so it is fine to just mock it @mock.patch( From 1094f0084c855258e486708dc6f1bb513063c71b Mon Sep 17 00:00:00 2001 From: ehab20011 Date: Mon, 1 Dec 2025 01:17:52 -0500 Subject: [PATCH 2/2] Add a Line to CHANGES.MD --- CHANGES.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGES.md b/CHANGES.md index 6ac873d9f26a..c2d1a04dfffb 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -85,6 +85,7 @@ ## Bugfixes * Fixed FirestoreV1 Beam connectors allow configuring inconsistent project/database IDs between RPC requests and routing headers #36895 (Java) ([#36895](https://github.com/apache/beam/issues/36895)). +* Fixed BigQuery client not receiving explicit project ID, causing failures in cross-project and impersonation scenarios (Python) ([#36857](https://github.com/apache/beam/issues/36857)). ## Known Issues