diff --git a/sdks/python/apache_beam/io/gcp/bigquery_read_internal.py b/sdks/python/apache_beam/io/gcp/bigquery_read_internal.py index f74b7dabfb7a..f3881ed261ae 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery_read_internal.py +++ b/sdks/python/apache_beam/io/gcp/bigquery_read_internal.py @@ -236,7 +236,7 @@ def _get_temp_dataset_id(self): else: raise ValueError("temp_dataset has to be either str or DatasetReference") - def setup(self): + def start_bundle(self): self.bq = bigquery_tools.BigQueryWrapper( temp_dataset_id=self._get_temp_dataset_id(), client=bigquery_tools.BigQueryWrapper._bigquery_client(self.options)) @@ -267,7 +267,7 @@ def process(self, table_reference.datasetId, table_reference.tableId) - def teardown(self): + def finish_bundle(self): if self.bq.created_temp_dataset: self.bq.clean_up_temporary_dataset(self._get_project())