diff --git a/airflow/providers/google/cloud/transfers/bigquery_to_sql.py b/airflow/providers/google/cloud/transfers/bigquery_to_sql.py index 736880726c94..e3a24fab5cc8 100644 --- a/airflow/providers/google/cloud/transfers/bigquery_to_sql.py +++ b/airflow/providers/google/cloud/transfers/bigquery_to_sql.py @@ -68,6 +68,8 @@ class BigQueryToSqlBaseOperator(BaseOperator): template_fields: Sequence[str] = ( "target_table_name", "impersonation_chain", + "dataset_id", + "table_id", ) def __init__( @@ -82,6 +84,8 @@ def __init__( batch_size: int = 1000, location: str | None = None, impersonation_chain: str | Sequence[str] | None = None, + dataset_id: str | None = None, + table_id: str | None = None, **kwargs, ) -> None: super().__init__(**kwargs) @@ -93,6 +97,8 @@ def __init__( self.batch_size = batch_size self.location = location self.impersonation_chain = impersonation_chain + self.dataset_id = dataset_id + self.table_id = table_id try: self.dataset_id, self.table_id = dataset_table.split(".") except ValueError: diff --git a/airflow/providers/google/cloud/utils/bigquery_get_data.py b/airflow/providers/google/cloud/utils/bigquery_get_data.py index d178aee963f1..6e1df55286dd 100644 --- a/airflow/providers/google/cloud/utils/bigquery_get_data.py +++ b/airflow/providers/google/cloud/utils/bigquery_get_data.py @@ -30,8 +30,8 @@ def bigquery_get_data( logger: Logger, - dataset_id: str, - table_id: str, + dataset_id: str | None, + table_id: str | None, big_query_hook: BigQueryHook, batch_size: int, selected_fields: list[str] | str | None,