diff --git a/airflow/providers/amazon/CHANGELOG.rst b/airflow/providers/amazon/CHANGELOG.rst index 1dedd33a81174..984b67e365d65 100644 --- a/airflow/providers/amazon/CHANGELOG.rst +++ b/airflow/providers/amazon/CHANGELOG.rst @@ -24,6 +24,16 @@ Changelog --------- +8.0.0 +...... + +Breaking changes +~~~~~~~~~~~~~~~~ + +.. warning:: + In this version of the provider, deprecated GCS hook's parameter ``delegate_to`` is removed from the following operators: ``GCSToS3Operator``, ``GlacierToGCSOperator`` and ``GoogleApiToS3Operator``. + Impersonation can be achieved instead by utilizing the ``impersonation_chain`` param. + 7.4.1 ..... diff --git a/airflow/providers/amazon/aws/transfers/gcs_to_s3.py b/airflow/providers/amazon/aws/transfers/gcs_to_s3.py index 6a4a58103689b..c9f6747436c52 100644 --- a/airflow/providers/amazon/aws/transfers/gcs_to_s3.py +++ b/airflow/providers/amazon/aws/transfers/gcs_to_s3.py @@ -47,9 +47,6 @@ class GCSToS3Operator(BaseOperator): :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud. :param google_cloud_storage_conn_id: (Deprecated) The connection ID used to connect to Google Cloud. This parameter has been deprecated. You should pass the gcp_conn_id parameter instead. - :param delegate_to: Google account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param dest_aws_conn_id: The destination S3 connection :param dest_s3_key: The base S3 key to be used to store the files. (templated) :param dest_verify: Whether or not to verify SSL certificates for S3 connection. @@ -101,7 +98,6 @@ def __init__( delimiter: str | None = None, gcp_conn_id: str = "google_cloud_default", google_cloud_storage_conn_id: str | None = None, - delegate_to: str | None = None, dest_aws_conn_id: str = "aws_default", dest_s3_key: str, dest_verify: str | bool | None = None, @@ -127,7 +123,6 @@ def __init__( self.prefix = prefix self.delimiter = delimiter self.gcp_conn_id = gcp_conn_id - self.delegate_to = delegate_to self.dest_aws_conn_id = dest_aws_conn_id self.dest_s3_key = dest_s3_key self.dest_verify = dest_verify @@ -141,7 +136,6 @@ def execute(self, context: Context) -> list[str]: # list all files in an Google Cloud Storage bucket hook = GCSHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.google_impersonation_chain, ) diff --git a/airflow/providers/amazon/aws/transfers/glacier_to_gcs.py b/airflow/providers/amazon/aws/transfers/glacier_to_gcs.py index 4a189230a4bb1..3814d3f44ff57 100644 --- a/airflow/providers/amazon/aws/transfers/glacier_to_gcs.py +++ b/airflow/providers/amazon/aws/transfers/glacier_to_gcs.py @@ -47,10 +47,7 @@ class GlacierToGCSOperator(BaseOperator): :param object_name: the name of the object to check in the Google cloud storage bucket. :param gzip: option to compress local file or file data for upload - :param chunk_size: size of chunk in bytes the that will downloaded from Glacier vault - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. + :param chunk_size: size of chunk in bytes the that will be downloaded from Glacier vault :param google_impersonation_chain: Optional Google service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -73,7 +70,6 @@ def __init__( object_name: str, gzip: bool, chunk_size: int = 1024, - delegate_to: str | None = None, google_impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -85,14 +81,12 @@ def __init__( self.object_name = object_name self.gzip = gzip self.chunk_size = chunk_size - self.delegate_to = delegate_to self.impersonation_chain = google_impersonation_chain def execute(self, context: Context) -> str: glacier_hook = GlacierHook(aws_conn_id=self.aws_conn_id) gcs_hook = GCSHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) job_id = glacier_hook.retrieve_inventory(vault_name=self.vault_name) diff --git a/airflow/providers/amazon/aws/transfers/google_api_to_s3.py b/airflow/providers/amazon/aws/transfers/google_api_to_s3.py index 7ac34242da31e..0a1bd7ba6f174 100644 --- a/airflow/providers/amazon/aws/transfers/google_api_to_s3.py +++ b/airflow/providers/amazon/aws/transfers/google_api_to_s3.py @@ -73,13 +73,10 @@ class GoogleApiToS3Operator(BaseOperator): .. note:: This means the response will be a list of responses. - :param google_api_num_retries: Define the number of retries for the google api requests being made + :param google_api_num_retries: Define the number of retries for the Google API requests being made if it fails. :param s3_overwrite: Specifies whether the s3 file will be overwritten if exists. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: Google account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param aws_conn_id: The connection id specifying the authentication information for the S3 Bucket. :param google_impersonation_chain: Optional Google service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token @@ -115,7 +112,6 @@ def __init__( google_api_num_retries: int = 0, s3_overwrite: bool = False, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, aws_conn_id: str = "aws_default", google_impersonation_chain: str | Sequence[str] | None = None, **kwargs, @@ -133,7 +129,6 @@ def __init__( self.google_api_num_retries = google_api_num_retries self.s3_overwrite = s3_overwrite self.gcp_conn_id = gcp_conn_id - self.delegate_to = delegate_to self.aws_conn_id = aws_conn_id self.google_impersonation_chain = google_impersonation_chain @@ -158,7 +153,6 @@ def execute(self, context: Context) -> None: def _retrieve_data_from_google_api(self) -> dict: google_discovery_api_hook = GoogleDiscoveryApiHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, api_service_name=self.google_api_service_name, api_version=self.google_api_service_version, impersonation_chain=self.google_impersonation_chain, diff --git a/airflow/providers/amazon/provider.yaml b/airflow/providers/amazon/provider.yaml index a6f27228049f0..294ea4e877a92 100644 --- a/airflow/providers/amazon/provider.yaml +++ b/airflow/providers/amazon/provider.yaml @@ -23,6 +23,7 @@ description: | suspended: false versions: + - 8.0.0 - 7.4.1 - 7.4.0 - 7.3.0 diff --git a/airflow/providers/apache/beam/CHANGELOG.rst b/airflow/providers/apache/beam/CHANGELOG.rst index 915f588768ff7..c9666e0bcd94d 100644 --- a/airflow/providers/apache/beam/CHANGELOG.rst +++ b/airflow/providers/apache/beam/CHANGELOG.rst @@ -24,6 +24,16 @@ Changelog --------- +5.0.0 +...... + +Breaking changes +~~~~~~~~~~~~~~~~ + +.. warning:: + In this version of the provider, deprecated GCS and Dataflow hooks' param ``delegate_to`` is removed from all Beam operators. + Impersonation can be achieved instead by utilizing the ``impersonation_chain`` param. + 4.3.0 ..... diff --git a/airflow/providers/apache/beam/operators/beam.py b/airflow/providers/apache/beam/operators/beam.py index e65aa32e6a320..258a89552d80c 100644 --- a/airflow/providers/apache/beam/operators/beam.py +++ b/airflow/providers/apache/beam/operators/beam.py @@ -22,7 +22,6 @@ import os import stat import tempfile -import warnings from abc import ABC, ABCMeta, abstractmethod from concurrent.futures import ThreadPoolExecutor, as_completed from contextlib import ExitStack @@ -57,7 +56,6 @@ class BeamDataflowMixin(metaclass=ABCMeta): dataflow_hook: DataflowHook | None dataflow_config: DataflowConfiguration gcp_conn_id: str - delegate_to: str | None dataflow_support_impersonation: bool = True def _set_dataflow( @@ -77,7 +75,6 @@ def _set_dataflow( def __set_dataflow_hook(self) -> DataflowHook: self.dataflow_hook = DataflowHook( gcp_conn_id=self.dataflow_config.gcp_conn_id or self.gcp_conn_id, - delegate_to=self.dataflow_config.delegate_to or self.delegate_to, poll_sleep=self.dataflow_config.poll_sleep, impersonation_chain=self.dataflow_config.impersonation_chain, drain_pipeline=self.dataflow_config.drain_pipeline, @@ -146,10 +143,6 @@ class BeamBasePipelineOperator(BaseOperator, BeamDataflowMixin, ABC): When defining labels (labels option), you can also provide a dictionary. :param gcp_conn_id: Optional. The connection ID to use connecting to Google Cloud Storage if python file is on GCS. - :param delegate_to: Optional. - The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param dataflow_config: Dataflow configuration, used when runner type is set to DataflowRunner, (optional) defaults to None. """ @@ -161,7 +154,6 @@ def __init__( default_pipeline_options: dict | None = None, pipeline_options: dict | None = None, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, dataflow_config: DataflowConfiguration | dict | None = None, **kwargs, ) -> None: @@ -170,11 +162,6 @@ def __init__( self.default_pipeline_options = default_pipeline_options or {} self.pipeline_options = pipeline_options or {} self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to if isinstance(dataflow_config, dict): self.dataflow_config = DataflowConfiguration(**dataflow_config) else: @@ -273,7 +260,6 @@ def __init__( py_requirements: list[str] | None = None, py_system_site_packages: bool = False, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, dataflow_config: DataflowConfiguration | dict | None = None, **kwargs, ) -> None: @@ -282,7 +268,6 @@ def __init__( default_pipeline_options=default_pipeline_options, pipeline_options=pipeline_options, gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, dataflow_config=dataflow_config, **kwargs, ) @@ -310,7 +295,7 @@ def execute(self, context: Context): with ExitStack() as exit_stack: if self.py_file.lower().startswith("gs://"): - gcs_hook = GCSHook(self.gcp_conn_id, self.delegate_to) + gcs_hook = GCSHook(gcp_conn_id=self.gcp_conn_id) tmp_gcs_file = exit_stack.enter_context(gcs_hook.provide_file(object_url=self.py_file)) self.py_file = tmp_gcs_file.name @@ -411,7 +396,6 @@ def __init__( default_pipeline_options: dict | None = None, pipeline_options: dict | None = None, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, dataflow_config: DataflowConfiguration | dict | None = None, **kwargs, ) -> None: @@ -420,7 +404,6 @@ def __init__( default_pipeline_options=default_pipeline_options, pipeline_options=pipeline_options, gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, dataflow_config=dataflow_config, **kwargs, ) @@ -441,7 +424,7 @@ def execute(self, context: Context): with ExitStack() as exit_stack: if self.jar.lower().startswith("gs://"): - gcs_hook = GCSHook(self.gcp_conn_id, self.delegate_to) + gcs_hook = GCSHook(self.gcp_conn_id) tmp_gcs_file = exit_stack.enter_context(gcs_hook.provide_file(object_url=self.jar)) self.jar = tmp_gcs_file.name @@ -568,7 +551,6 @@ def __init__( default_pipeline_options: dict | None = None, pipeline_options: dict | None = None, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, dataflow_config: DataflowConfiguration | dict | None = None, **kwargs, ) -> None: @@ -577,7 +559,6 @@ def __init__( default_pipeline_options=default_pipeline_options, pipeline_options=pipeline_options, gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, dataflow_config=dataflow_config, **kwargs, ) @@ -620,7 +601,7 @@ def execute(self, context: Context): with ExitStack() as exit_stack: if go_artifact.is_located_on_gcs(): - gcs_hook = GCSHook(self.gcp_conn_id, self.delegate_to) + gcs_hook = GCSHook(self.gcp_conn_id) tmp_dir = exit_stack.enter_context(tempfile.TemporaryDirectory(prefix="apache-beam-go")) go_artifact.download_from_gcs(gcs_hook=gcs_hook, tmp_dir=tmp_dir) diff --git a/airflow/providers/apache/beam/provider.yaml b/airflow/providers/apache/beam/provider.yaml index cb1d6e7440291..aa355a2c5e753 100644 --- a/airflow/providers/apache/beam/provider.yaml +++ b/airflow/providers/apache/beam/provider.yaml @@ -23,6 +23,7 @@ description: | suspended: false versions: + - 5.0.0 - 4.3.0 - 4.2.0 - 4.1.1 diff --git a/airflow/providers/google/CHANGELOG.rst b/airflow/providers/google/CHANGELOG.rst index 516c67c46c908..0dd710999b74d 100644 --- a/airflow/providers/google/CHANGELOG.rst +++ b/airflow/providers/google/CHANGELOG.rst @@ -33,6 +33,11 @@ Google has announced sunset of Campaign Manager 360 v3.5 by Apr 20, 2023. For mo please check: ``_ . As a result, the default api version for Campaign Manager 360 operator was updated to the latest v4 version. +.. warning:: + In this version of the provider, deprecated ``delegate_to`` param is removed from all GCP operators, hooks, and triggers, as well as from firestore and gsuite + transfer operators that interact with GCS. Impersonation can be achieved instead by utilizing the ``impersonation_chain`` param. + The ``delegate_to`` param will still be available only in gsuite and marketing platform hooks and operators, that don't interact with Google Cloud. + 9.0.0 ..... diff --git a/airflow/providers/google/cloud/hooks/automl.py b/airflow/providers/google/cloud/hooks/automl.py index ed9e29c870775..6e7e5aa005746 100644 --- a/airflow/providers/google/cloud/hooks/automl.py +++ b/airflow/providers/google/cloud/hooks/automl.py @@ -24,7 +24,6 @@ """ from __future__ import annotations -import warnings from typing import Sequence from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault @@ -65,16 +64,16 @@ class CloudAutoMLHook(GoogleBaseHook): def __init__( self, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" ) super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) self._client: AutoMlClient | None = None diff --git a/airflow/providers/google/cloud/hooks/bigquery.py b/airflow/providers/google/cloud/hooks/bigquery.py index e25918d083301..e182090ee16ae 100644 --- a/airflow/providers/google/cloud/hooks/bigquery.py +++ b/airflow/providers/google/cloud/hooks/bigquery.py @@ -74,7 +74,6 @@ class BigQueryHook(GoogleBaseHook, DbApiHook): Interact with BigQuery. This hook uses the Google Cloud connection. :param gcp_conn_id: The Airflow connection used for GCP credentials. - :param delegate_to: This performs a task on one host with reference to other hosts. :param use_legacy_sql: This specifies whether to use legacy SQL dialect. :param location: The location of the BigQuery resource. :param api_resource_configs: This contains params configuration applied for Google BigQuery jobs. @@ -91,20 +90,20 @@ class BigQueryHook(GoogleBaseHook, DbApiHook): def __init__( self, gcp_conn_id: str = GoogleBaseHook.default_conn_name, - delegate_to: str | None = None, use_legacy_sql: bool = True, location: str | None = None, api_resource_configs: dict | None = None, impersonation_chain: str | Sequence[str] | None = None, labels: dict | None = None, + **kwargs, ) -> None: - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" ) super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) self.use_legacy_sql = use_legacy_sql diff --git a/airflow/providers/google/cloud/hooks/bigquery_dts.py b/airflow/providers/google/cloud/hooks/bigquery_dts.py index 044e89e6b5f65..c2dff601d25e0 100644 --- a/airflow/providers/google/cloud/hooks/bigquery_dts.py +++ b/airflow/providers/google/cloud/hooks/bigquery_dts.py @@ -18,7 +18,6 @@ """This module contains a BigQuery Hook.""" from __future__ import annotations -import warnings from copy import copy from typing import Sequence @@ -58,17 +57,17 @@ class BiqQueryDataTransferServiceHook(GoogleBaseHook): def __init__( self, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, location: str | None = None, impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" ) super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) self.location = location @@ -282,13 +281,17 @@ class AsyncBiqQueryDataTransferServiceHook(GoogleBaseAsyncHook): def __init__( self, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, location: str | None = None, impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ): + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" + ) super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, location=location, impersonation_chain=impersonation_chain, ) diff --git a/airflow/providers/google/cloud/hooks/bigtable.py b/airflow/providers/google/cloud/hooks/bigtable.py index 93120f3d075e5..c51c1a990febb 100644 --- a/airflow/providers/google/cloud/hooks/bigtable.py +++ b/airflow/providers/google/cloud/hooks/bigtable.py @@ -19,7 +19,6 @@ from __future__ import annotations import enum -import warnings from typing import Sequence from google.cloud.bigtable import Client, enums @@ -43,16 +42,16 @@ class BigtableHook(GoogleBaseHook): def __init__( self, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" ) super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) self._client: Client | None = None diff --git a/airflow/providers/google/cloud/hooks/cloud_build.py b/airflow/providers/google/cloud/hooks/cloud_build.py index 826bc6bcbebfe..a88935810b324 100644 --- a/airflow/providers/google/cloud/hooks/cloud_build.py +++ b/airflow/providers/google/cloud/hooks/cloud_build.py @@ -18,7 +18,6 @@ """Hook for Google Cloud Build service.""" from __future__ import annotations -import warnings from typing import Sequence from google.api_core.client_options import ClientOptions @@ -42,9 +41,6 @@ class CloudBuildHook(GoogleBaseHook): Hook for the Google Cloud Build Service. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -58,16 +54,15 @@ class CloudBuildHook(GoogleBaseHook): def __init__( self, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" ) - super().__init__( - gcp_conn_id=gcp_conn_id, delegate_to=delegate_to, impersonation_chain=impersonation_chain - ) + super().__init__(gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain) self._client: dict[str, CloudBuildClient] = {} def _get_build_id_from_operation(self, operation: Operation) -> str: @@ -628,6 +623,14 @@ def update_build_trigger( class CloudBuildAsyncHook(GoogleBaseHook): """Asynchronous Hook for the Google Cloud Build Service.""" + def __init__(self, **kwargs): + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" + ) + super().__init__(**kwargs) + @GoogleBaseHook.fallback_to_default_project_id async def get_cloud_build( self, diff --git a/airflow/providers/google/cloud/hooks/cloud_composer.py b/airflow/providers/google/cloud/hooks/cloud_composer.py index 793aec9f20d27..5c994b3a58501 100644 --- a/airflow/providers/google/cloud/hooks/cloud_composer.py +++ b/airflow/providers/google/cloud/hooks/cloud_composer.py @@ -46,6 +46,14 @@ class CloudComposerHook(GoogleBaseHook): client_options = ClientOptions(api_endpoint="composer.googleapis.com:443") + def __init__(self, **kwargs): + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" + ) + super().__init__(**kwargs) + def get_environment_client(self) -> EnvironmentsClient: """Retrieves client library object that allow access Environments service.""" return EnvironmentsClient( @@ -284,6 +292,14 @@ def list_image_versions( class CloudComposerAsyncHook(GoogleBaseHook): """Hook for Google Cloud Composer async APIs.""" + def __init__(self, **kwargs): + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" + ) + super().__init__(**kwargs) + client_options = ClientOptions(api_endpoint="composer.googleapis.com:443") def get_environment_client(self) -> EnvironmentsAsyncClient: diff --git a/airflow/providers/google/cloud/hooks/cloud_memorystore.py b/airflow/providers/google/cloud/hooks/cloud_memorystore.py index a3dffba6b8d68..e25b4bd71d2c7 100644 --- a/airflow/providers/google/cloud/hooks/cloud_memorystore.py +++ b/airflow/providers/google/cloud/hooks/cloud_memorystore.py @@ -27,7 +27,6 @@ """ from __future__ import annotations -import warnings from typing import Sequence from google.api_core import path_template @@ -58,9 +57,6 @@ class CloudMemorystoreHook(GoogleBaseHook): keyword arguments rather than positional. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -74,16 +70,16 @@ class CloudMemorystoreHook(GoogleBaseHook): def __init__( self, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" ) super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) self._client: CloudRedisClient | None = None @@ -499,9 +495,6 @@ class CloudMemorystoreMemcachedHook(GoogleBaseHook): keyword arguments rather than positional. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -515,12 +508,10 @@ class CloudMemorystoreMemcachedHook(GoogleBaseHook): def __init__( self, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, ) -> None: super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) self._client: CloudMemcacheClient | None = None diff --git a/airflow/providers/google/cloud/hooks/cloud_sql.py b/airflow/providers/google/cloud/hooks/cloud_sql.py index edd34e43d7040..4b430fa02ed11 100644 --- a/airflow/providers/google/cloud/hooks/cloud_sql.py +++ b/airflow/providers/google/cloud/hooks/cloud_sql.py @@ -31,7 +31,6 @@ import subprocess import time import uuid -import warnings from inspect import signature from pathlib import Path from subprocess import PIPE, Popen @@ -80,7 +79,6 @@ class CloudSQLHook(GoogleBaseHook): :param api_version: This is the version of the api. :param gcp_conn_id: The Airflow connection used for GCP credentials. - :param delegate_to: This performs a task on one host with reference to other hosts. :param impersonation_chain: This is the optional service account to impersonate using short term credentials. """ @@ -94,16 +92,16 @@ def __init__( self, api_version: str, gcp_conn_id: str = default_conn_name, - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" ) super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) self.api_version = api_version diff --git a/airflow/providers/google/cloud/hooks/cloud_storage_transfer_service.py b/airflow/providers/google/cloud/hooks/cloud_storage_transfer_service.py index 9d251f34b0a0d..dc447ae052f7b 100644 --- a/airflow/providers/google/cloud/hooks/cloud_storage_transfer_service.py +++ b/airflow/providers/google/cloud/hooks/cloud_storage_transfer_service.py @@ -127,16 +127,16 @@ def __init__( self, api_version: str = "v1", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" ) super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) self.api_version = api_version diff --git a/airflow/providers/google/cloud/hooks/compute.py b/airflow/providers/google/cloud/hooks/compute.py index cf1b11d6d5d72..8b839c595a49f 100644 --- a/airflow/providers/google/cloud/hooks/compute.py +++ b/airflow/providers/google/cloud/hooks/compute.py @@ -19,7 +19,6 @@ from __future__ import annotations import time -import warnings from typing import Any, Sequence from google.api_core.retry import Retry @@ -56,16 +55,16 @@ def __init__( self, api_version: str = "v1", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" ) super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) self.api_version = api_version diff --git a/airflow/providers/google/cloud/hooks/compute_ssh.py b/airflow/providers/google/cloud/hooks/compute_ssh.py index d0f8825004490..7b474070aee37 100644 --- a/airflow/providers/google/cloud/hooks/compute_ssh.py +++ b/airflow/providers/google/cloud/hooks/compute_ssh.py @@ -18,7 +18,6 @@ import shlex import time -import warnings from io import StringIO from typing import Any @@ -80,9 +79,6 @@ class ComputeEngineSSHHook(SSHHook): keys are managed using instance metadata :param expire_time: The maximum amount of time in seconds before the private key expires :param gcp_conn_id: The connection id to use when fetching connection information - :param delegate_to: The account to impersonate, if any. - For this to work, the service account making the request must have - domain-wide delegation enabled. """ conn_name_attr = "gcp_conn_id" @@ -109,8 +105,13 @@ def __init__( use_iap_tunnel: bool = False, use_oslogin: bool = True, expire_time: int = 300, - delegate_to: str | None = None, + **kwargs, ) -> None: + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" + ) # Ignore original constructor # super().__init__() self.instance_name = instance_name @@ -123,20 +124,15 @@ def __init__( self.use_oslogin = use_oslogin self.expire_time = expire_time self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self._conn: Any | None = None @cached_property def _oslogin_hook(self) -> OSLoginHook: - return OSLoginHook(gcp_conn_id=self.gcp_conn_id, delegate_to=self.delegate_to) + return OSLoginHook(gcp_conn_id=self.gcp_conn_id) @cached_property def _compute_hook(self) -> ComputeEngineHook: - return ComputeEngineHook(gcp_conn_id=self.gcp_conn_id, delegate_to=self.delegate_to) + return ComputeEngineHook(gcp_conn_id=self.gcp_conn_id) def _load_connection_config(self): def _boolify(value): diff --git a/airflow/providers/google/cloud/hooks/datacatalog.py b/airflow/providers/google/cloud/hooks/datacatalog.py index 4049ae3ed98d5..2dd60931fe42c 100644 --- a/airflow/providers/google/cloud/hooks/datacatalog.py +++ b/airflow/providers/google/cloud/hooks/datacatalog.py @@ -16,7 +16,6 @@ # under the License. from __future__ import annotations -import warnings from typing import Sequence from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault @@ -44,9 +43,6 @@ class CloudDataCatalogHook(GoogleBaseHook): Hook for Google Cloud Data Catalog Service. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -60,16 +56,16 @@ class CloudDataCatalogHook(GoogleBaseHook): def __init__( self, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" ) super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) self._client: DataCatalogClient | None = None diff --git a/airflow/providers/google/cloud/hooks/dataflow.py b/airflow/providers/google/cloud/hooks/dataflow.py index 3f7fd4b54ecc9..d69aa305434ef 100644 --- a/airflow/providers/google/cloud/hooks/dataflow.py +++ b/airflow/providers/google/cloud/hooks/dataflow.py @@ -516,26 +516,26 @@ class DataflowHook(GoogleBaseHook): def __init__( self, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, poll_sleep: int = 10, impersonation_chain: str | Sequence[str] | None = None, drain_pipeline: bool = False, cancel_timeout: int | None = 5 * 60, wait_until_finished: bool | None = None, + **kwargs, ) -> None: + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" + ) self.poll_sleep = poll_sleep self.drain_pipeline = drain_pipeline self.cancel_timeout = cancel_timeout self.wait_until_finished = wait_until_finished self.job_id: str | None = None self.beam_hook = BeamHook(BeamRunnerType.DataflowRunner) - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) @@ -1189,6 +1189,14 @@ class AsyncDataflowHook(GoogleBaseAsyncHook): sync_hook_class = DataflowHook + def __init__(self, **kwargs): + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" + ) + super().__init__(**kwargs) + async def initialize_client(self, client_class): """ Initialize object of the given class. diff --git a/airflow/providers/google/cloud/hooks/dataform.py b/airflow/providers/google/cloud/hooks/dataform.py index aa8506c16ed7a..8905f2546d261 100644 --- a/airflow/providers/google/cloud/hooks/dataform.py +++ b/airflow/providers/google/cloud/hooks/dataform.py @@ -38,6 +38,14 @@ class DataformHook(GoogleBaseHook): """Hook for Google Cloud DataForm APIs.""" + def __init__(self, **kwargs): + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" + ) + super().__init__(**kwargs) + def get_dataform_client(self) -> DataformClient: """Retrieves client library object that allow access to Cloud Dataform service.""" return DataformClient(credentials=self.get_credentials()) diff --git a/airflow/providers/google/cloud/hooks/datafusion.py b/airflow/providers/google/cloud/hooks/datafusion.py index 286964426b5d5..935b28fed4c3d 100644 --- a/airflow/providers/google/cloud/hooks/datafusion.py +++ b/airflow/providers/google/cloud/hooks/datafusion.py @@ -19,7 +19,6 @@ import json import os -import warnings from time import monotonic, sleep from typing import Any, Dict, Sequence from urllib.parse import quote, urlencode, urljoin @@ -67,16 +66,16 @@ def __init__( self, api_version: str = "v1beta1", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" ) super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) self.api_version = api_version @@ -490,6 +489,14 @@ class DataFusionAsyncHook(GoogleBaseAsyncHook): sync_hook_class = DataFusionHook scopes = ["https://www.googleapis.com/auth/cloud-platform"] + def __init__(self, **kwargs): + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" + ) + super().__init__(**kwargs) + @staticmethod def _base_url(instance_url: str, namespace: str) -> str: return urljoin(f"{instance_url}/", f"v3/namespaces/{quote(namespace)}/apps/") diff --git a/airflow/providers/google/cloud/hooks/dataplex.py b/airflow/providers/google/cloud/hooks/dataplex.py index 2acfe3064c645..405870fd319b5 100644 --- a/airflow/providers/google/cloud/hooks/dataplex.py +++ b/airflow/providers/google/cloud/hooks/dataplex.py @@ -17,7 +17,6 @@ """This module contains Google Dataplex hook.""" from __future__ import annotations -import warnings from typing import Any, Sequence from google.api_core.client_options import ClientOptions @@ -39,8 +38,6 @@ class DataplexHook(GoogleBaseHook): :param api_version: The version of the api that will be requested for example 'v3'. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate, if any. For this to work, the service accountmaking the - request must have domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -57,16 +54,16 @@ def __init__( self, api_version: str = "v1", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" ) super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) self.api_version = api_version diff --git a/airflow/providers/google/cloud/hooks/dataproc.py b/airflow/providers/google/cloud/hooks/dataproc.py index 4974a162980c6..625193439047a 100644 --- a/airflow/providers/google/cloud/hooks/dataproc.py +++ b/airflow/providers/google/cloud/hooks/dataproc.py @@ -20,7 +20,6 @@ import time import uuid -import warnings from typing import Any, Sequence from google.api_core.client_options import ClientOptions @@ -209,14 +208,15 @@ class DataprocHook(GoogleBaseHook): def __init__( self, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" ) - super().__init__(gcp_conn_id, delegate_to, impersonation_chain) + super().__init__(gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain) def get_cluster_client(self, region: str | None = None) -> ClusterControllerClient: """Returns ClusterControllerClient.""" @@ -1060,10 +1060,15 @@ class DataprocAsyncHook(GoogleBaseHook): def __init__( self, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: - super().__init__(gcp_conn_id, delegate_to, impersonation_chain) + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" + ) + super().__init__(gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain) self._cached_client: JobControllerAsyncClient | None = None def get_cluster_client(self, region: str | None = None) -> ClusterControllerAsyncClient: diff --git a/airflow/providers/google/cloud/hooks/dataproc_metastore.py b/airflow/providers/google/cloud/hooks/dataproc_metastore.py index c7dcebee9fcab..5eb0f7689b10e 100644 --- a/airflow/providers/google/cloud/hooks/dataproc_metastore.py +++ b/airflow/providers/google/cloud/hooks/dataproc_metastore.py @@ -37,6 +37,14 @@ class DataprocMetastoreHook(GoogleBaseHook): """Hook for Google Cloud Dataproc Metastore APIs.""" + def __init__(self, **kwargs): + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" + ) + super().__init__(**kwargs) + def get_dataproc_metastore_client(self) -> DataprocMetastoreClient: """Returns DataprocMetastoreClient.""" client_options = ClientOptions(api_endpoint="metastore.googleapis.com:443") diff --git a/airflow/providers/google/cloud/hooks/datastore.py b/airflow/providers/google/cloud/hooks/datastore.py index 87fd9afcc19da..8e14fe96bbb74 100644 --- a/airflow/providers/google/cloud/hooks/datastore.py +++ b/airflow/providers/google/cloud/hooks/datastore.py @@ -19,7 +19,6 @@ from __future__ import annotations import time -import warnings from typing import Any, Sequence from googleapiclient.discovery import Resource, build @@ -40,17 +39,17 @@ class DatastoreHook(GoogleBaseHook): def __init__( self, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, api_version: str = "v1", impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" ) super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) self.connection = None diff --git a/airflow/providers/google/cloud/hooks/dlp.py b/airflow/providers/google/cloud/hooks/dlp.py index b7a71f5dd1dc2..6a8fb216309f7 100644 --- a/airflow/providers/google/cloud/hooks/dlp.py +++ b/airflow/providers/google/cloud/hooks/dlp.py @@ -28,7 +28,6 @@ import re import time -import warnings from typing import Sequence from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault @@ -73,9 +72,6 @@ class CloudDLPHook(GoogleBaseHook): on Google Cloud based data sets. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -89,16 +85,16 @@ class CloudDLPHook(GoogleBaseHook): def __init__( self, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" ) super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) self._client: DlpServiceClient | None = None diff --git a/airflow/providers/google/cloud/hooks/functions.py b/airflow/providers/google/cloud/hooks/functions.py index 084d6fb06ab3b..8409b5a8eadf0 100644 --- a/airflow/providers/google/cloud/hooks/functions.py +++ b/airflow/providers/google/cloud/hooks/functions.py @@ -19,7 +19,6 @@ from __future__ import annotations import time -import warnings from typing import Sequence import requests @@ -46,16 +45,16 @@ def __init__( self, api_version: str, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" ) super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) self.api_version = api_version diff --git a/airflow/providers/google/cloud/hooks/gcs.py b/airflow/providers/google/cloud/hooks/gcs.py index a0b94ec665c9f..742b05e32e50a 100644 --- a/airflow/providers/google/cloud/hooks/gcs.py +++ b/airflow/providers/google/cloud/hooks/gcs.py @@ -23,7 +23,6 @@ import os import shutil import time -import warnings from contextlib import contextmanager from datetime import datetime from functools import partial @@ -152,16 +151,17 @@ class GCSHook(GoogleBaseHook): def __init__( self, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" ) + super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) diff --git a/airflow/providers/google/cloud/hooks/gdm.py b/airflow/providers/google/cloud/hooks/gdm.py index 007c30e93af9d..7d228c8c95efc 100644 --- a/airflow/providers/google/cloud/hooks/gdm.py +++ b/airflow/providers/google/cloud/hooks/gdm.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import warnings from typing import Any, Sequence from googleapiclient.discovery import Resource, build @@ -35,16 +34,16 @@ class GoogleDeploymentManagerHook(GoogleBaseHook): def __init__( self, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" ) super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) diff --git a/airflow/providers/google/cloud/hooks/kms.py b/airflow/providers/google/cloud/hooks/kms.py index 553a38578bd57..6dfa85335b849 100644 --- a/airflow/providers/google/cloud/hooks/kms.py +++ b/airflow/providers/google/cloud/hooks/kms.py @@ -19,7 +19,6 @@ from __future__ import annotations import base64 -import warnings from typing import Sequence from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault @@ -45,9 +44,6 @@ class CloudKMSHook(GoogleBaseHook): Hook for Google Cloud Key Management service. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -61,16 +57,16 @@ class CloudKMSHook(GoogleBaseHook): def __init__( self, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" ) super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) self._conn: KeyManagementServiceClient | None = None diff --git a/airflow/providers/google/cloud/hooks/kubernetes_engine.py b/airflow/providers/google/cloud/hooks/kubernetes_engine.py index 704988f52d3a3..1b67779e0571d 100644 --- a/airflow/providers/google/cloud/hooks/kubernetes_engine.py +++ b/airflow/providers/google/cloud/hooks/kubernetes_engine.py @@ -73,13 +73,17 @@ class GKEHook(GoogleBaseHook): def __init__( self, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, location: str | None = None, impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" + ) super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) self._client: ClusterManagerClient | None = None @@ -303,13 +307,11 @@ class GKEAsyncHook(GoogleBaseAsyncHook): def __init__( self, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, location: str | None = None, impersonation_chain: str | Sequence[str] | None = None, ) -> None: super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) self._client: ClusterManagerAsyncClient | None = None diff --git a/airflow/providers/google/cloud/hooks/life_sciences.py b/airflow/providers/google/cloud/hooks/life_sciences.py index 24ad1792acf7e..186c959a88ccc 100644 --- a/airflow/providers/google/cloud/hooks/life_sciences.py +++ b/airflow/providers/google/cloud/hooks/life_sciences.py @@ -19,7 +19,6 @@ from __future__ import annotations import time -import warnings from typing import Sequence import google.api_core.path_template @@ -41,9 +40,6 @@ class LifeSciencesHook(GoogleBaseHook): :param api_version: API version used (for example v1 or v1beta1). :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -60,16 +56,16 @@ def __init__( self, api_version: str = "v2beta", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" ) super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) self.api_version = api_version diff --git a/airflow/providers/google/cloud/hooks/mlengine.py b/airflow/providers/google/cloud/hooks/mlengine.py index 03da6a282af1b..583c1fe0a4e63 100644 --- a/airflow/providers/google/cloud/hooks/mlengine.py +++ b/airflow/providers/google/cloud/hooks/mlengine.py @@ -84,6 +84,14 @@ class MLEngineHook(GoogleBaseHook): keyword arguments rather than positional. """ + def __init__(self, **kwargs): + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" + ) + super().__init__(**kwargs) + def get_conn(self) -> Resource: """ Retrieves the connection to MLEngine. diff --git a/airflow/providers/google/cloud/hooks/natural_language.py b/airflow/providers/google/cloud/hooks/natural_language.py index 7fa5f965118bc..db0e0528ec58e 100644 --- a/airflow/providers/google/cloud/hooks/natural_language.py +++ b/airflow/providers/google/cloud/hooks/natural_language.py @@ -18,7 +18,6 @@ """This module contains a Google Cloud Natural Language Hook.""" from __future__ import annotations -import warnings from typing import Sequence from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault @@ -44,9 +43,6 @@ class CloudNaturalLanguageHook(GoogleBaseHook): Hook for Google Cloud Natural Language Service. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -60,16 +56,16 @@ class CloudNaturalLanguageHook(GoogleBaseHook): def __init__( self, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" ) super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) self._conn = None diff --git a/airflow/providers/google/cloud/hooks/os_login.py b/airflow/providers/google/cloud/hooks/os_login.py index 38c8001890833..02ddadb5999d2 100644 --- a/airflow/providers/google/cloud/hooks/os_login.py +++ b/airflow/providers/google/cloud/hooks/os_login.py @@ -22,7 +22,6 @@ """ from __future__ import annotations -import warnings from typing import Sequence from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault @@ -44,16 +43,16 @@ class OSLoginHook(GoogleBaseHook): def __init__( self, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" ) super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) self._conn: OsLoginServiceClient | None = None diff --git a/airflow/providers/google/cloud/hooks/pubsub.py b/airflow/providers/google/cloud/hooks/pubsub.py index 76cb70689a94a..35beacc02d4d7 100644 --- a/airflow/providers/google/cloud/hooks/pubsub.py +++ b/airflow/providers/google/cloud/hooks/pubsub.py @@ -67,16 +67,16 @@ class PubSubHook(GoogleBaseHook): def __init__( self, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" ) super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) self._client = None diff --git a/airflow/providers/google/cloud/hooks/secret_manager.py b/airflow/providers/google/cloud/hooks/secret_manager.py index 25dce82d454c8..1a6d143d84eba 100644 --- a/airflow/providers/google/cloud/hooks/secret_manager.py +++ b/airflow/providers/google/cloud/hooks/secret_manager.py @@ -18,7 +18,6 @@ """Hook for Secrets Manager service""" from __future__ import annotations -import warnings from typing import Sequence from airflow.providers.google.cloud._internal_client.secret_manager_client import _SecretManagerClient @@ -35,9 +34,6 @@ class SecretsManagerHook(GoogleBaseHook): keyword arguments rather than positional. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. (Deprecated) :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -51,16 +47,16 @@ class SecretsManagerHook(GoogleBaseHook): def __init__( self, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" ) super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) self.client = _SecretManagerClient(credentials=self.get_credentials()) diff --git a/airflow/providers/google/cloud/hooks/spanner.py b/airflow/providers/google/cloud/hooks/spanner.py index 54e2ec7fae8f9..60da28e9bc323 100644 --- a/airflow/providers/google/cloud/hooks/spanner.py +++ b/airflow/providers/google/cloud/hooks/spanner.py @@ -18,7 +18,6 @@ """This module contains a Google Cloud Spanner Hook.""" from __future__ import annotations -import warnings from typing import Callable, Sequence from google.api_core.exceptions import AlreadyExists, GoogleAPICallError @@ -44,16 +43,16 @@ class SpannerHook(GoogleBaseHook): def __init__( self, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" ) super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) self._client = None diff --git a/airflow/providers/google/cloud/hooks/speech_to_text.py b/airflow/providers/google/cloud/hooks/speech_to_text.py index f344fcdeca639..e21bb9677c390 100644 --- a/airflow/providers/google/cloud/hooks/speech_to_text.py +++ b/airflow/providers/google/cloud/hooks/speech_to_text.py @@ -18,7 +18,6 @@ """This module contains a Google Cloud Speech Hook.""" from __future__ import annotations -import warnings from typing import Sequence from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault @@ -35,9 +34,6 @@ class CloudSpeechToTextHook(GoogleBaseHook): Hook for Google Cloud Speech API. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -51,16 +47,16 @@ class CloudSpeechToTextHook(GoogleBaseHook): def __init__( self, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" ) super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) self._client = None diff --git a/airflow/providers/google/cloud/hooks/stackdriver.py b/airflow/providers/google/cloud/hooks/stackdriver.py index 41af0991ad2d8..3a5738e3d507c 100644 --- a/airflow/providers/google/cloud/hooks/stackdriver.py +++ b/airflow/providers/google/cloud/hooks/stackdriver.py @@ -19,7 +19,6 @@ from __future__ import annotations import json -import warnings from typing import Any, Sequence from google.api_core.exceptions import InvalidArgument @@ -40,16 +39,16 @@ class StackdriverHook(GoogleBaseHook): def __init__( self, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" ) super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) self._policy_client = None diff --git a/airflow/providers/google/cloud/hooks/tasks.py b/airflow/providers/google/cloud/hooks/tasks.py index 57d66bc5536bf..1301a3406e76a 100644 --- a/airflow/providers/google/cloud/hooks/tasks.py +++ b/airflow/providers/google/cloud/hooks/tasks.py @@ -22,7 +22,6 @@ """ from __future__ import annotations -import warnings from typing import Sequence from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault @@ -45,9 +44,6 @@ class CloudTasksHook(GoogleBaseHook): keyword arguments rather than positional. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -61,16 +57,16 @@ class CloudTasksHook(GoogleBaseHook): def __init__( self, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" ) super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) self._client: CloudTasksClient | None = None diff --git a/airflow/providers/google/cloud/hooks/text_to_speech.py b/airflow/providers/google/cloud/hooks/text_to_speech.py index 0370d148f5ac3..8cd44fc64e9b9 100644 --- a/airflow/providers/google/cloud/hooks/text_to_speech.py +++ b/airflow/providers/google/cloud/hooks/text_to_speech.py @@ -18,7 +18,6 @@ """This module contains a Google Cloud Text to Speech Hook.""" from __future__ import annotations -import warnings from typing import Sequence from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault @@ -43,9 +42,6 @@ class CloudTextToSpeechHook(GoogleBaseHook): keyword arguments rather than positional. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -59,16 +55,16 @@ class CloudTextToSpeechHook(GoogleBaseHook): def __init__( self, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" ) super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) self._client: TextToSpeechClient | None = None diff --git a/airflow/providers/google/cloud/hooks/translate.py b/airflow/providers/google/cloud/hooks/translate.py index e9e655e269209..6025030379000 100644 --- a/airflow/providers/google/cloud/hooks/translate.py +++ b/airflow/providers/google/cloud/hooks/translate.py @@ -18,7 +18,6 @@ """This module contains a Google Cloud Translate Hook.""" from __future__ import annotations -import warnings from typing import Sequence from google.cloud.translate_v2 import Client @@ -38,16 +37,16 @@ class CloudTranslateHook(GoogleBaseHook): def __init__( self, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" ) super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) self._client: Client | None = None diff --git a/airflow/providers/google/cloud/hooks/vertex_ai/auto_ml.py b/airflow/providers/google/cloud/hooks/vertex_ai/auto_ml.py index ead3dd2858465..d0bb7ea4fdf0e 100644 --- a/airflow/providers/google/cloud/hooks/vertex_ai/auto_ml.py +++ b/airflow/providers/google/cloud/hooks/vertex_ai/auto_ml.py @@ -74,16 +74,16 @@ class AutoMLHook(GoogleBaseHook): def __init__( self, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" ) super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) self._job: None | ( diff --git a/airflow/providers/google/cloud/hooks/vertex_ai/batch_prediction_job.py b/airflow/providers/google/cloud/hooks/vertex_ai/batch_prediction_job.py index 982c4c4dd753a..93cd6902bba75 100644 --- a/airflow/providers/google/cloud/hooks/vertex_ai/batch_prediction_job.py +++ b/airflow/providers/google/cloud/hooks/vertex_ai/batch_prediction_job.py @@ -26,7 +26,6 @@ """ from __future__ import annotations -import warnings from typing import Sequence from google.api_core.client_options import ClientOptions @@ -47,16 +46,16 @@ class BatchPredictionJobHook(GoogleBaseHook): def __init__( self, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" ) super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) self._batch_prediction_job: BatchPredictionJob | None = None diff --git a/airflow/providers/google/cloud/hooks/vertex_ai/custom_job.py b/airflow/providers/google/cloud/hooks/vertex_ai/custom_job.py index da5b014283771..04d1347ed5875 100644 --- a/airflow/providers/google/cloud/hooks/vertex_ai/custom_job.py +++ b/airflow/providers/google/cloud/hooks/vertex_ai/custom_job.py @@ -18,7 +18,6 @@ """This module contains a Google Cloud Vertex AI hook.""" from __future__ import annotations -import warnings from typing import Sequence from google.api_core.client_options import ClientOptions @@ -51,16 +50,16 @@ class CustomJobHook(GoogleBaseHook): def __init__( self, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" ) super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) self._job: None | ( diff --git a/airflow/providers/google/cloud/hooks/vertex_ai/dataset.py b/airflow/providers/google/cloud/hooks/vertex_ai/dataset.py index aadc0733bb804..ac898d7776913 100644 --- a/airflow/providers/google/cloud/hooks/vertex_ai/dataset.py +++ b/airflow/providers/google/cloud/hooks/vertex_ai/dataset.py @@ -41,6 +41,14 @@ class DatasetHook(GoogleBaseHook): """Hook for Google Cloud Vertex AI Dataset APIs.""" + def __init__(self, **kwargs): + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" + ) + super().__init__(**kwargs) + def get_dataset_service_client(self, region: str | None = None) -> DatasetServiceClient: """Returns DatasetServiceClient.""" if region and region != "global": diff --git a/airflow/providers/google/cloud/hooks/vertex_ai/endpoint_service.py b/airflow/providers/google/cloud/hooks/vertex_ai/endpoint_service.py index 6bee752463d16..f3adb69f1c35b 100644 --- a/airflow/providers/google/cloud/hooks/vertex_ai/endpoint_service.py +++ b/airflow/providers/google/cloud/hooks/vertex_ai/endpoint_service.py @@ -46,6 +46,14 @@ class EndpointServiceHook(GoogleBaseHook): """Hook for Google Cloud Vertex AI Endpoint Service APIs.""" + def __init__(self, **kwargs): + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" + ) + super().__init__(**kwargs) + def get_endpoint_service_client(self, region: str | None = None) -> EndpointServiceClient: """Returns EndpointServiceClient.""" if region and region != "global": diff --git a/airflow/providers/google/cloud/hooks/vertex_ai/hyperparameter_tuning_job.py b/airflow/providers/google/cloud/hooks/vertex_ai/hyperparameter_tuning_job.py index ad8b119f8d6e7..e51ccb79163d2 100644 --- a/airflow/providers/google/cloud/hooks/vertex_ai/hyperparameter_tuning_job.py +++ b/airflow/providers/google/cloud/hooks/vertex_ai/hyperparameter_tuning_job.py @@ -27,7 +27,6 @@ """ from __future__ import annotations -import warnings from typing import Sequence from google.api_core.client_options import ClientOptions @@ -48,16 +47,16 @@ class HyperparameterTuningJobHook(GoogleBaseHook): def __init__( self, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" ) super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) self._hyperparameter_tuning_job: HyperparameterTuningJob | None = None diff --git a/airflow/providers/google/cloud/hooks/vertex_ai/model_service.py b/airflow/providers/google/cloud/hooks/vertex_ai/model_service.py index ca7f6aca85a89..9f7e53b9574ae 100644 --- a/airflow/providers/google/cloud/hooks/vertex_ai/model_service.py +++ b/airflow/providers/google/cloud/hooks/vertex_ai/model_service.py @@ -41,6 +41,14 @@ class ModelServiceHook(GoogleBaseHook): """Hook for Google Cloud Vertex AI Endpoint Service APIs.""" + def __init__(self, **kwargs): + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" + ) + super().__init__(**kwargs) + def get_model_service_client(self, region: str | None = None) -> ModelServiceClient: """Returns ModelServiceClient.""" if region and region != "global": diff --git a/airflow/providers/google/cloud/hooks/video_intelligence.py b/airflow/providers/google/cloud/hooks/video_intelligence.py index 1e20f8863cfa2..103025c77f96b 100644 --- a/airflow/providers/google/cloud/hooks/video_intelligence.py +++ b/airflow/providers/google/cloud/hooks/video_intelligence.py @@ -18,7 +18,6 @@ """This module contains a Google Cloud Video Intelligence Hook.""" from __future__ import annotations -import warnings from typing import Sequence from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault @@ -39,9 +38,6 @@ class CloudVideoIntelligenceHook(GoogleBaseHook): keyword arguments rather than positional. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -55,16 +51,16 @@ class CloudVideoIntelligenceHook(GoogleBaseHook): def __init__( self, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" ) super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) self._conn = None diff --git a/airflow/providers/google/cloud/hooks/vision.py b/airflow/providers/google/cloud/hooks/vision.py index f6fc612956fba..dace3619cff4f 100644 --- a/airflow/providers/google/cloud/hooks/vision.py +++ b/airflow/providers/google/cloud/hooks/vision.py @@ -18,7 +18,6 @@ """This module contains a Google Cloud Vision Hook.""" from __future__ import annotations -import warnings from copy import deepcopy from typing import Any, Callable, Sequence @@ -127,16 +126,16 @@ class CloudVisionHook(GoogleBaseHook): def __init__( self, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" ) super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) self._client = None diff --git a/airflow/providers/google/cloud/hooks/workflows.py b/airflow/providers/google/cloud/hooks/workflows.py index a8a99252c77c5..6e1f6ed1209eb 100644 --- a/airflow/providers/google/cloud/hooks/workflows.py +++ b/airflow/providers/google/cloud/hooks/workflows.py @@ -39,6 +39,14 @@ class WorkflowsHook(GoogleBaseHook): keyword arguments rather than positional. """ + def __init__(self, **kwargs): + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" + ) + super().__init__(**kwargs) + def get_workflows_client(self) -> WorkflowsClient: """Returns WorkflowsClient.""" return WorkflowsClient(credentials=self.get_credentials(), client_info=CLIENT_INFO) diff --git a/airflow/providers/google/cloud/operators/bigquery.py b/airflow/providers/google/cloud/operators/bigquery.py index 092290a77fa30..2464a1658b596 100644 --- a/airflow/providers/google/cloud/operators/bigquery.py +++ b/airflow/providers/google/cloud/operators/bigquery.py @@ -811,9 +811,6 @@ class BigQueryGetDataOperator(GoogleCloudBaseOperator): :param deferrable: Run operator in the deferrable mode :param poll_interval: (Deferrable mode only) polling period in seconds to check for the status of job. Defaults to 4 seconds. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. Deprecated. """ template_fields: Sequence[str] = ( @@ -838,7 +835,6 @@ def __init__( location: str | None = None, impersonation_chain: str | Sequence[str] | None = None, deferrable: bool = False, - delegate_to: str | None = None, poll_interval: float = 4.0, **kwargs, ) -> None: @@ -849,11 +845,6 @@ def __init__( self.max_results = int(max_results) self.selected_fields = selected_fields self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.location = location self.impersonation_chain = impersonation_chain self.project_id = project_id @@ -892,7 +883,6 @@ def generate_query(self) -> str: def execute(self, context: Context): hook = BigQueryHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) self.hook = hook @@ -977,9 +967,6 @@ class BigQueryExecuteQueryOperator(GoogleCloudBaseOperator): must be ``true`` if this is set to ``false``. For standard SQL queries, this flag is ignored and results are never flattened. :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param udf_config: The User Defined Function configuration for the query. See https://cloud.google.com/bigquery/user-defined-functions for details. :param use_legacy_sql: Whether to use legacy SQL (true) or standard SQL (false). @@ -1060,7 +1047,6 @@ def __init__( allow_large_results: bool = False, flatten_results: bool | None = None, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, udf_config: list | None = None, use_legacy_sql: bool = True, maximum_billing_tier: int | None = None, @@ -1092,7 +1078,6 @@ def __init__( self.allow_large_results = allow_large_results self.flatten_results = flatten_results self.gcp_conn_id = gcp_conn_id - self.delegate_to = delegate_to self.udf_config = udf_config self.use_legacy_sql = use_legacy_sql self.maximum_billing_tier = maximum_billing_tier @@ -1115,7 +1100,6 @@ def execute(self, context: Context): self.hook = BigQueryHook( gcp_conn_id=self.gcp_conn_id, use_legacy_sql=self.use_legacy_sql, - delegate_to=self.delegate_to, location=self.location, impersonation_chain=self.impersonation_chain, ) @@ -1214,9 +1198,6 @@ class BigQueryCreateEmptyTableOperator(GoogleCloudBaseOperator): interact with the Bigquery service. :param google_cloud_storage_conn_id: [Optional] The connection ID used to connect to Google Cloud. and interact with the Google Cloud Storage service. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param labels: a dictionary containing labels for the table, passed to BigQuery **Example (with schema JSON in GCS)**: :: @@ -1326,7 +1307,6 @@ def __init__( cluster_fields: list[str] | None = None, impersonation_chain: str | Sequence[str] | None = None, if_exists: str = "log", - delegate_to: str | None = None, bigquery_conn_id: str | None = None, exists_ok: bool | None = None, **kwargs, @@ -1348,11 +1328,6 @@ def __init__( self.gcs_schema_object = gcs_schema_object self.gcp_conn_id = gcp_conn_id self.google_cloud_storage_conn_id = google_cloud_storage_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.time_partitioning = {} if time_partitioning is None else time_partitioning self.labels = labels self.view = view @@ -1371,7 +1346,6 @@ def __init__( def execute(self, context: Context) -> None: bq_hook = BigQueryHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, location=self.location, impersonation_chain=self.impersonation_chain, ) @@ -1380,7 +1354,6 @@ def execute(self, context: Context) -> None: gcs_bucket, gcs_object = _parse_gcs_url(self.gcs_schema_object) gcs_hook = GCSHook( gcp_conn_id=self.google_cloud_storage_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) schema_fields_string = gcs_hook.download_as_byte_array(gcs_bucket, gcs_object).decode("utf-8") @@ -1482,9 +1455,6 @@ class BigQueryCreateExternalTableOperator(GoogleCloudBaseOperator): interact with the Bigquery service. :param google_cloud_storage_conn_id: (Optional) The connection ID used to connect to Google Cloud and interact with the Google Cloud Storage service. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param src_fmt_configs: configure optional fields specific to the source format :param labels: a dictionary containing labels for the table, passed to BigQuery :param encryption_configuration: [Optional] Custom encryption configuration (e.g., Cloud KMS keys). @@ -1542,7 +1512,6 @@ def __init__( encryption_configuration: dict | None = None, location: str | None = None, impersonation_chain: str | Sequence[str] | None = None, - delegate_to: str | None = None, bigquery_conn_id: str | None = None, **kwargs, ) -> None: @@ -1625,11 +1594,6 @@ def __init__( self.allow_jagged_rows = allow_jagged_rows self.gcp_conn_id = gcp_conn_id self.google_cloud_storage_conn_id = google_cloud_storage_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.autodetect = autodetect self.src_fmt_configs = src_fmt_configs or {} @@ -1641,7 +1605,6 @@ def __init__( def execute(self, context: Context) -> None: bq_hook = BigQueryHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, location=self.location, impersonation_chain=self.impersonation_chain, ) @@ -1661,7 +1624,6 @@ def execute(self, context: Context) -> None: if not self.schema_fields and self.schema_object and self.source_format != "DATASTORE_BACKUP": gcs_hook = GCSHook( gcp_conn_id=self.google_cloud_storage_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) schema_fields = json.loads(gcs_hook.download(self.bucket, self.schema_object).decode("utf-8")) @@ -1730,9 +1692,6 @@ class BigQueryDeleteDatasetOperator(GoogleCloudBaseOperator): Will raise HttpError 400: "{dataset_id} is still in use" if set to False and dataset is not empty. The default value is False. :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -1768,18 +1727,12 @@ def __init__( delete_contents: bool = False, gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, - delegate_to: str | None = None, **kwargs, ) -> None: self.dataset_id = dataset_id self.project_id = project_id self.delete_contents = delete_contents self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain super().__init__(**kwargs) @@ -1789,7 +1742,6 @@ def execute(self, context: Context) -> None: bq_hook = BigQueryHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) @@ -1814,9 +1766,6 @@ class BigQueryCreateEmptyDatasetOperator(GoogleCloudBaseOperator): More info: https://cloud.google.com/bigquery/docs/reference/rest/v2/datasets#resource :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -1860,7 +1809,6 @@ def __init__( gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, if_exists: str = "log", - delegate_to: str | None = None, exists_ok: bool | None = None, **kwargs, ) -> None: @@ -1870,11 +1818,6 @@ def __init__( self.location = location self.gcp_conn_id = gcp_conn_id self.dataset_reference = dataset_reference if dataset_reference else {} - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain if exists_ok is not None: warnings.warn("`exists_ok` parameter is deprecated, please use `if_exists`", DeprecationWarning) @@ -1887,7 +1830,6 @@ def __init__( def execute(self, context: Context) -> None: bq_hook = BigQueryHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, location=self.location, impersonation_chain=self.impersonation_chain, ) @@ -1930,9 +1872,6 @@ class BigQueryGetDatasetOperator(GoogleCloudBaseOperator): :param project_id: The name of the project where we want to create the dataset. Don't need to provide, if projectId in dataset_reference. :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -1958,24 +1897,17 @@ def __init__( project_id: str | None = None, gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, - delegate_to: str | None = None, **kwargs, ) -> None: self.dataset_id = dataset_id self.project_id = project_id self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain super().__init__(**kwargs) def execute(self, context: Context): bq_hook = BigQueryHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) @@ -2005,9 +1937,6 @@ class BigQueryGetDatasetTablesOperator(GoogleCloudBaseOperator): self.project_id will be used. :param max_results: (Optional) the maximum number of tables to return. :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -2033,25 +1962,18 @@ def __init__( max_results: int | None = None, gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, - delegate_to: str | None = None, **kwargs, ) -> None: self.dataset_id = dataset_id self.project_id = project_id self.max_results = max_results self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain super().__init__(**kwargs) def execute(self, context: Context): bq_hook = BigQueryHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) @@ -2077,9 +1999,6 @@ class BigQueryPatchDatasetOperator(GoogleCloudBaseOperator): :param project_id: The name of the project where we want to create the dataset. Don't need to provide, if projectId in dataset_reference. :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -2106,7 +2025,6 @@ def __init__( project_id: str | None = None, gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, - delegate_to: str | None = None, **kwargs, ) -> None: warnings.warn( @@ -2118,18 +2036,12 @@ def __init__( self.project_id = project_id self.gcp_conn_id = gcp_conn_id self.dataset_resource = dataset_resource - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain super().__init__(**kwargs) def execute(self, context: Context): bq_hook = BigQueryHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) @@ -2161,9 +2073,6 @@ class BigQueryUpdateTableOperator(GoogleCloudBaseOperator): :param project_id: The name of the project where we want to create the table. Don't need to provide, if projectId in table_reference. :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -2194,7 +2103,6 @@ def __init__( project_id: str | None = None, gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, - delegate_to: str | None = None, **kwargs, ) -> None: self.dataset_id = dataset_id @@ -2203,18 +2111,12 @@ def __init__( self.fields = fields self.gcp_conn_id = gcp_conn_id self.table_resource = table_resource - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain super().__init__(**kwargs) def execute(self, context: Context): bq_hook = BigQueryHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) @@ -2257,9 +2159,6 @@ class BigQueryUpdateDatasetOperator(GoogleCloudBaseOperator): :param project_id: The name of the project where we want to create the dataset. Don't need to provide, if projectId in dataset_reference. :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -2288,7 +2187,6 @@ def __init__( project_id: str | None = None, gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, - delegate_to: str | None = None, **kwargs, ) -> None: self.dataset_id = dataset_id @@ -2296,18 +2194,12 @@ def __init__( self.fields = fields self.gcp_conn_id = gcp_conn_id self.dataset_resource = dataset_resource - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain super().__init__(**kwargs) def execute(self, context: Context): bq_hook = BigQueryHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) fields = self.fields or list(self.dataset_resource.keys()) @@ -2341,9 +2233,6 @@ class BigQueryDeleteTableOperator(GoogleCloudBaseOperator): ``(.|:).`` that indicates which table will be deleted. (templated) :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param ignore_if_missing: if True, then return success even if the requested table does not exist. :param location: The location used for the operation. @@ -2371,18 +2260,12 @@ def __init__( ignore_if_missing: bool = False, location: str | None = None, impersonation_chain: str | Sequence[str] | None = None, - delegate_to: str | None = None, **kwargs, ) -> None: super().__init__(**kwargs) self.deletion_dataset_table = deletion_dataset_table self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.ignore_if_missing = ignore_if_missing self.location = location self.impersonation_chain = impersonation_chain @@ -2391,7 +2274,6 @@ def execute(self, context: Context) -> None: self.log.info("Deleting: %s", self.deletion_dataset_table) hook = BigQueryHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, location=self.location, impersonation_chain=self.impersonation_chain, ) @@ -2414,9 +2296,6 @@ class BigQueryUpsertTableOperator(GoogleCloudBaseOperator): :param project_id: The name of the project where we want to update the dataset. Don't need to provide, if projectId in dataset_reference. :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud. - :param delegate_to: The account to impersonate, if any. - For this to work, the service account making the request must have domain-wide - delegation enabled. :param location: The location used for the operation. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token @@ -2447,7 +2326,6 @@ def __init__( gcp_conn_id: str = "google_cloud_default", location: str | None = None, impersonation_chain: str | Sequence[str] | None = None, - delegate_to: str | None = None, **kwargs, ) -> None: super().__init__(**kwargs) @@ -2456,11 +2334,6 @@ def __init__( self.table_resource = table_resource self.project_id = project_id self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.location = location self.impersonation_chain = impersonation_chain @@ -2468,7 +2341,6 @@ def execute(self, context: Context) -> None: self.log.info("Upserting Dataset: %s with table_resource: %s", self.dataset_id, self.table_resource) hook = BigQueryHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, location=self.location, impersonation_chain=self.impersonation_chain, ) @@ -2522,9 +2394,6 @@ class BigQueryUpdateTableSchemaOperator(GoogleCloudBaseOperator): :param project_id: The name of the project where we want to update the dataset. Don't need to provide, if projectId in dataset_reference. :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud. - :param delegate_to: The account to impersonate, if any. - For this to work, the service account making the request must have domain-wide - delegation enabled. :param location: The location used for the operation. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token @@ -2557,7 +2426,6 @@ def __init__( project_id: str | None = None, gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, - delegate_to: str | None = None, **kwargs, ) -> None: self.schema_fields_updates = schema_fields_updates @@ -2566,18 +2434,12 @@ def __init__( self.dataset_id = dataset_id self.project_id = project_id self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain super().__init__(**kwargs) def execute(self, context: Context): bq_hook = BigQueryHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) @@ -2636,9 +2498,6 @@ class BigQueryInsertJobOperator(GoogleCloudBaseOperator): :param project_id: Google Cloud Project where the job is running :param location: location the job is running :param gcp_conn_id: The connection ID used to connect to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -2684,7 +2543,6 @@ def __init__( result_timeout: float | None = None, deferrable: bool = False, poll_interval: float = 4.0, - delegate_to: str | None = None, **kwargs, ) -> None: super().__init__(**kwargs) @@ -2693,11 +2551,6 @@ def __init__( self.job_id = job_id self.project_id = project_id self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.force_rerun = force_rerun self.reattach_states: set[str] = reattach_states or set() self.impersonation_chain = impersonation_chain @@ -2738,7 +2591,6 @@ def _handle_job_error(job: BigQueryJob) -> None: def execute(self, context: Any): hook = BigQueryHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) self.hook = hook diff --git a/airflow/providers/google/cloud/operators/cloud_build.py b/airflow/providers/google/cloud/operators/cloud_build.py index 17a06a7acee2d..88d9b434bb544 100644 --- a/airflow/providers/google/cloud/operators/cloud_build.py +++ b/airflow/providers/google/cloud/operators/cloud_build.py @@ -20,7 +20,6 @@ import json import re -import warnings from copy import deepcopy from typing import TYPE_CHECKING, Any, Sequence from urllib.parse import unquote, urlsplit @@ -155,9 +154,6 @@ class CloudBuildCreateBuildOperator(GoogleCloudBaseOperator): If set as a sequence, the identities from the list must grant Service Account Token Creator IAM role to the directly preceding identity, with first account from the list granting this role to the originating account (templated). - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param retry: Designation of what errors, if any, should be retried. :param timeout: The timeout for this request. :param metadata: Strings which should be sent along with the request as metadata. @@ -179,7 +175,6 @@ def __init__( metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, - delegate_to: str | None = None, poll_interval: float = 4.0, deferrable: bool = False, location: str = "global", @@ -196,11 +191,6 @@ def __init__( self.metadata = metadata self.gcp_conn_id = gcp_conn_id self.impersonation_chain = impersonation_chain - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.poll_interval = poll_interval self.deferrable = deferrable self.location = location @@ -219,7 +209,6 @@ def execute(self, context: Context): hook = CloudBuildHook( gcp_conn_id=self.gcp_conn_id, impersonation_chain=self.impersonation_chain, - delegate_to=self.delegate_to, ) build = BuildProcessor(build=self.build).process_body() @@ -244,7 +233,6 @@ def execute(self, context: Context): project_id=self.project_id, gcp_conn_id=self.gcp_conn_id, impersonation_chain=self.impersonation_chain, - delegate_to=self.delegate_to, poll_interval=self.poll_interval, location=self.location, ), @@ -270,7 +258,6 @@ def execute_complete(self, context: Context, event: dict): hook = CloudBuildHook( gcp_conn_id=self.gcp_conn_id, impersonation_chain=self.impersonation_chain, - delegate_to=self.delegate_to, ) self.log.info("Cloud Build completed with response %s ", event["message"]) project_id = self.project_id or hook.project_id diff --git a/airflow/providers/google/cloud/operators/cloud_composer.py b/airflow/providers/google/cloud/operators/cloud_composer.py index dee1a635ad05c..5cfc06d0c9b5c 100644 --- a/airflow/providers/google/cloud/operators/cloud_composer.py +++ b/airflow/providers/google/cloud/operators/cloud_composer.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import warnings from typing import TYPE_CHECKING, Sequence from google.api_core.exceptions import AlreadyExists @@ -106,9 +105,6 @@ class CloudComposerCreateEnvironmentOperator(GoogleCloudBaseOperator): If set as a sequence, the identities from the list must grant Service Account Token Creator IAM role to the directly preceding identity, with first account from the list granting this role to the originating account (templated). - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param retry: Designation of what errors, if any, should be retried. :param timeout: The timeout for this request. :param metadata: Strings which should be sent along with the request as metadata. @@ -136,7 +132,6 @@ def __init__( environment: Environment | dict, gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, - delegate_to: str | None = None, retry: Retry | _MethodDefault = DEFAULT, timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), @@ -154,11 +149,6 @@ def __init__( self.metadata = metadata self.gcp_conn_id = gcp_conn_id self.impersonation_chain = impersonation_chain - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.deferrable = deferrable self.pooling_period_seconds = pooling_period_seconds @@ -166,7 +156,6 @@ def execute(self, context: Context): hook = CloudComposerHook( gcp_conn_id=self.gcp_conn_id, impersonation_chain=self.impersonation_chain, - delegate_to=self.delegate_to, ) name = hook.get_environment_name(self.project_id, self.region, self.environment_id) @@ -198,7 +187,6 @@ def execute(self, context: Context): operation_name=result.operation.name, gcp_conn_id=self.gcp_conn_id, impersonation_chain=self.impersonation_chain, - delegate_to=self.delegate_to, pooling_period_seconds=self.pooling_period_seconds, ), method_name=GOOGLE_DEFAULT_DEFERRABLE_METHOD_NAME, @@ -219,7 +207,6 @@ def execute_complete(self, context: Context, event: dict): hook = CloudComposerHook( gcp_conn_id=self.gcp_conn_id, impersonation_chain=self.impersonation_chain, - delegate_to=self.delegate_to, ) env = hook.get_environment( @@ -254,12 +241,9 @@ class CloudComposerDeleteEnvironmentOperator(GoogleCloudBaseOperator): If set as a sequence, the identities from the list must grant Service Account Token Creator IAM role to the directly preceding identity, with first account from the list granting this role to the originating account (templated). - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param deferrable: Run operator in the deferrable mode :param pooling_period_seconds: Optional: Control the rate of the poll for the result of deferrable run. - By default the trigger will poll every 30 seconds. + By default, the trigger will poll every 30 seconds. """ template_fields = ( @@ -280,7 +264,6 @@ def __init__( metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, - delegate_to: str | None = None, deferrable: bool = False, pooling_period_seconds: int = 30, **kwargs, @@ -294,11 +277,6 @@ def __init__( self.metadata = metadata self.gcp_conn_id = gcp_conn_id self.impersonation_chain = impersonation_chain - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.deferrable = deferrable self.pooling_period_seconds = pooling_period_seconds @@ -306,7 +284,6 @@ def execute(self, context: Context): hook = CloudComposerHook( gcp_conn_id=self.gcp_conn_id, impersonation_chain=self.impersonation_chain, - delegate_to=self.delegate_to, ) result = hook.delete_environment( project_id=self.project_id, @@ -326,7 +303,6 @@ def execute(self, context: Context): operation_name=result.operation.name, gcp_conn_id=self.gcp_conn_id, impersonation_chain=self.impersonation_chain, - delegate_to=self.delegate_to, pooling_period_seconds=self.pooling_period_seconds, ), method_name=GOOGLE_DEFAULT_DEFERRABLE_METHOD_NAME, @@ -355,9 +331,6 @@ class CloudComposerGetEnvironmentOperator(GoogleCloudBaseOperator): If set as a sequence, the identities from the list must grant Service Account Token Creator IAM role to the directly preceding identity, with first account from the list granting this role to the originating account (templated). - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. """ template_fields = ( @@ -380,7 +353,6 @@ def __init__( metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, - delegate_to: str | None = None, **kwargs, ) -> None: super().__init__(**kwargs) @@ -392,17 +364,11 @@ def __init__( self.metadata = metadata self.gcp_conn_id = gcp_conn_id self.impersonation_chain = impersonation_chain - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to def execute(self, context: Context): hook = CloudComposerHook( gcp_conn_id=self.gcp_conn_id, impersonation_chain=self.impersonation_chain, - delegate_to=self.delegate_to, ) result = hook.get_environment( @@ -439,9 +405,6 @@ class CloudComposerListEnvironmentsOperator(GoogleCloudBaseOperator): If set as a sequence, the identities from the list must grant Service Account Token Creator IAM role to the directly preceding identity, with first account from the list granting this role to the originating account (templated). - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. """ template_fields = ( @@ -464,7 +427,6 @@ def __init__( metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, - delegate_to: str | None = None, **kwargs, ) -> None: super().__init__(**kwargs) @@ -477,17 +439,11 @@ def __init__( self.metadata = metadata self.gcp_conn_id = gcp_conn_id self.impersonation_chain = impersonation_chain - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to def execute(self, context: Context): hook = CloudComposerHook( gcp_conn_id=self.gcp_conn_id, impersonation_chain=self.impersonation_chain, - delegate_to=self.delegate_to, ) CloudComposerEnvironmentsLink.persist(operator_instance=self, context=context) result = hook.list_environments( @@ -526,12 +482,9 @@ class CloudComposerUpdateEnvironmentOperator(GoogleCloudBaseOperator): If set as a sequence, the identities from the list must grant Service Account Token Creator IAM role to the directly preceding identity, with first account from the list granting this role to the originating account (templated). - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param deferrable: Run operator in the deferrable mode :param pooling_period_seconds: Optional: Control the rate of the poll for the result of deferrable run. - By default the trigger will poll every 30 seconds. + By default, the trigger will poll every 30 seconds. """ template_fields = ( @@ -556,7 +509,6 @@ def __init__( metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, - delegate_to: str | None = None, deferrable: bool = False, pooling_period_seconds: int = 30, **kwargs, @@ -572,11 +524,6 @@ def __init__( self.metadata = metadata self.gcp_conn_id = gcp_conn_id self.impersonation_chain = impersonation_chain - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.deferrable = deferrable self.pooling_period_seconds = pooling_period_seconds @@ -584,7 +531,6 @@ def execute(self, context: Context): hook = CloudComposerHook( gcp_conn_id=self.gcp_conn_id, impersonation_chain=self.impersonation_chain, - delegate_to=self.delegate_to, ) result = hook.update_environment( @@ -610,7 +556,6 @@ def execute(self, context: Context): operation_name=result.operation.name, gcp_conn_id=self.gcp_conn_id, impersonation_chain=self.impersonation_chain, - delegate_to=self.delegate_to, pooling_period_seconds=self.pooling_period_seconds, ), method_name=GOOGLE_DEFAULT_DEFERRABLE_METHOD_NAME, @@ -621,7 +566,6 @@ def execute_complete(self, context: Context, event: dict): hook = CloudComposerHook( gcp_conn_id=self.gcp_conn_id, impersonation_chain=self.impersonation_chain, - delegate_to=self.delegate_to, ) env = hook.get_environment( @@ -654,9 +598,6 @@ class CloudComposerListImageVersionsOperator(GoogleCloudBaseOperator): If set as a sequence, the identities from the list must grant Service Account Token Creator IAM role to the directly preceding identity, with first account from the list granting this role to the originating account (templated). - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. """ template_fields = ( @@ -678,7 +619,6 @@ def __init__( metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, - delegate_to: str | None = None, **kwargs, ) -> None: super().__init__(**kwargs) @@ -692,17 +632,11 @@ def __init__( self.metadata = metadata self.gcp_conn_id = gcp_conn_id self.impersonation_chain = impersonation_chain - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to def execute(self, context: Context): hook = CloudComposerHook( gcp_conn_id=self.gcp_conn_id, impersonation_chain=self.impersonation_chain, - delegate_to=self.delegate_to, ) result = hook.list_image_versions( project_id=self.project_id, diff --git a/airflow/providers/google/cloud/operators/cloud_storage_transfer_service.py b/airflow/providers/google/cloud/operators/cloud_storage_transfer_service.py index 6e737fcca0cb3..ec8ce7d8e1cbe 100644 --- a/airflow/providers/google/cloud/operators/cloud_storage_transfer_service.py +++ b/airflow/providers/google/cloud/operators/cloud_storage_transfer_service.py @@ -18,7 +18,6 @@ """This module contains Google Cloud Transfer operators.""" from __future__ import annotations -import warnings from copy import deepcopy from datetime import date, time from typing import TYPE_CHECKING, Sequence @@ -795,9 +794,6 @@ class CloudDataTransferServiceS3ToGCSOperator(GoogleCloudBaseOperator): :param aws_conn_id: The source S3 connection :param gcp_conn_id: The destination connection ID to use when connecting to Google Cloud Storage. - :param delegate_to: Google account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param description: Optional transfer service job description :param schedule: Optional transfer service schedule; If not set, run transfer job once as soon as the operator runs @@ -849,7 +845,6 @@ def __init__( project_id: str | None = None, aws_conn_id: str = "aws_default", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, description: str | None = None, schedule: dict | None = None, object_conditions: dict | None = None, @@ -869,11 +864,6 @@ def __init__( self.project_id = project_id self.aws_conn_id = aws_conn_id self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.description = description self.schedule = schedule self.object_conditions = object_conditions @@ -891,7 +881,6 @@ def _validate_inputs(self) -> None: def execute(self, context: Context) -> None: hook = CloudDataTransferServiceHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.google_impersonation_chain, ) body = self._create_body() @@ -971,9 +960,6 @@ class CloudDataTransferServiceGCSToGCSOperator(GoogleCloudBaseOperator): owns the job :param gcp_conn_id: Optional connection ID to use when connecting to Google Cloud Storage. - :param delegate_to: Google account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param description: Optional transfer service job description :param schedule: Optional transfer service schedule; If not set, run transfer job once as soon as the operator runs @@ -1024,7 +1010,6 @@ def __init__( destination_path: str | None = None, project_id: str | None = None, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, description: str | None = None, schedule: dict | None = None, object_conditions: dict | None = None, @@ -1043,11 +1028,6 @@ def __init__( self.destination_path = destination_path self.project_id = project_id self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.description = description self.schedule = schedule self.object_conditions = object_conditions @@ -1065,7 +1045,6 @@ def _validate_inputs(self) -> None: def execute(self, context: Context) -> None: hook = CloudDataTransferServiceHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.google_impersonation_chain, ) diff --git a/airflow/providers/google/cloud/operators/dataflow.py b/airflow/providers/google/cloud/operators/dataflow.py index df40cdb9abc91..9ac2c41fce7b2 100644 --- a/airflow/providers/google/cloud/operators/dataflow.py +++ b/airflow/providers/google/cloud/operators/dataflow.py @@ -70,8 +70,6 @@ class DataflowConfiguration: If set to None or missing, the default project_id from the Google Cloud connection is used. :param location: Job location. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, if any. - For this to work, the service account making the request must have domain-wide delegation enabled. :param poll_sleep: The time in seconds to sleep between polling Google Cloud Platform for the dataflow job status while the job is in the JOB_STATE_RUNNING state. @@ -140,7 +138,6 @@ def __init__( project_id: str | None = None, location: str | None = DEFAULT_DATAFLOW_LOCATION, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, poll_sleep: int = 10, impersonation_chain: str | Sequence[str] | None = None, drain_pipeline: bool = False, @@ -155,11 +152,6 @@ def __init__( self.project_id = project_id self.location = location self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.poll_sleep = poll_sleep self.impersonation_chain = impersonation_chain self.drain_pipeline = drain_pipeline @@ -243,9 +235,6 @@ class DataflowCreateJavaJobOperator(GoogleCloudBaseOperator): If set to None or missing, the default project_id from the Google Cloud connection is used. :param location: Job location. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param poll_sleep: The time in seconds to sleep between polling Google Cloud Platform for the dataflow job status while the job is in the JOB_STATE_RUNNING state. @@ -346,7 +335,6 @@ def __init__( project_id: str | None = None, location: str = DEFAULT_DATAFLOW_LOCATION, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, poll_sleep: int = 10, job_class: str | None = None, check_if_running: CheckJobRunning = CheckJobRunning.WaitForRun, @@ -372,11 +360,6 @@ def __init__( self.project_id = project_id self.location = location self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.jar = jar self.multiple_jobs = multiple_jobs self.job_name = job_name @@ -396,7 +379,6 @@ def execute(self, context: Context): self.beam_hook = BeamHook(runner=BeamRunnerType.DataflowRunner) self.dataflow_hook = DataflowHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, poll_sleep=self.poll_sleep, cancel_timeout=self.cancel_timeout, wait_until_finished=self.wait_until_finished, @@ -422,7 +404,7 @@ def set_current_job_id(job_id): with ExitStack() as exit_stack: if self.jar.lower().startswith("gs://"): - gcs_hook = GCSHook(self.gcp_conn_id, self.delegate_to) + gcs_hook = GCSHook(self.gcp_conn_id) tmp_gcs_file = exit_stack.enter_context(gcs_hook.provide_file(object_url=self.jar)) self.jar = tmp_gcs_file.name @@ -490,9 +472,6 @@ class DataflowTemplatedJobStartOperator(GoogleCloudBaseOperator): If set to None or missing, the default project_id from the Google Cloud connection is used. :param location: Job location. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param poll_sleep: The time in seconds to sleep between polling Google Cloud Platform for the dataflow job status while the job is in the JOB_STATE_RUNNING state. @@ -622,7 +601,6 @@ def __init__( parameters: dict[str, str] | None = None, location: str = DEFAULT_DATAFLOW_LOCATION, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, poll_sleep: int = 10, impersonation_chain: str | Sequence[str] | None = None, environment: dict | None = None, @@ -642,11 +620,6 @@ def __init__( self.project_id = project_id self.location = location self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.poll_sleep = poll_sleep self.impersonation_chain = impersonation_chain self.environment = environment @@ -674,7 +647,6 @@ def _validate_deferrable_params(self): def hook(self) -> DataflowHook: hook = DataflowHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, poll_sleep=self.poll_sleep, impersonation_chain=self.impersonation_chain, cancel_timeout=self.cancel_timeout, @@ -719,7 +691,6 @@ def set_current_job(current_job): job_id=job_id, location=self.location, gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, poll_sleep=self.poll_sleep, impersonation_chain=self.impersonation_chain, cancel_timeout=self.cancel_timeout, @@ -763,9 +734,6 @@ class DataflowStartFlexTemplateOperator(GoogleCloudBaseOperator): :param project_id: The ID of the GCP project that owns the job. :param gcp_conn_id: The connection ID to use connecting to Google Cloud Platform. - :param delegate_to: The account to impersonate, if any. - For this to work, the service account making the request must have - domain-wide delegation enabled. :param drain_pipeline: Optional, set to True if want to stop streaming job by draining it instead of canceling during killing task instance. See: https://cloud.google.com/dataflow/docs/guides/stopping-a-pipeline @@ -824,7 +792,6 @@ def __init__( location: str, project_id: str | None = None, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, drain_pipeline: bool = False, cancel_timeout: int | None = 10 * 60, wait_until_finished: bool | None = None, @@ -838,11 +805,6 @@ def __init__( self.location = location self.project_id = project_id self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.drain_pipeline = drain_pipeline self.cancel_timeout = cancel_timeout self.wait_until_finished = wait_until_finished @@ -867,7 +829,6 @@ def _validate_deferrable_params(self): def hook(self) -> DataflowHook: hook = DataflowHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, drain_pipeline=self.drain_pipeline, cancel_timeout=self.cancel_timeout, wait_until_finished=self.wait_until_finished, @@ -904,7 +865,6 @@ def set_current_job(current_job): job_id=job_id, location=self.location, gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, cancel_timeout=self.cancel_timeout, ), @@ -966,9 +926,6 @@ class DataflowStartSqlJobOperator(GoogleCloudBaseOperator): If set to ``None`` or missing, the default project_id from the GCP connection is used. :param gcp_conn_id: The connection ID to use connecting to Google Cloud Platform. - :param delegate_to: The account to impersonate, if any. - For this to work, the service account making the request must have - domain-wide delegation enabled. :param drain_pipeline: Optional, set to True if want to stop streaming job by draining it instead of canceling during killing task instance. See: https://cloud.google.com/dataflow/docs/guides/stopping-a-pipeline @@ -1000,7 +957,6 @@ def __init__( location: str = DEFAULT_DATAFLOW_LOCATION, project_id: str | None = None, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, drain_pipeline: bool = False, impersonation_chain: str | Sequence[str] | None = None, *args, @@ -1013,11 +969,6 @@ def __init__( self.location = location self.project_id = project_id self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.drain_pipeline = drain_pipeline self.impersonation_chain = impersonation_chain self.job = None @@ -1026,7 +977,6 @@ def __init__( def execute(self, context: Context): self.hook = DataflowHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, drain_pipeline=self.drain_pipeline, impersonation_chain=self.impersonation_chain, ) @@ -1111,9 +1061,6 @@ class DataflowCreatePythonJobOperator(GoogleCloudBaseOperator): :param project_id: Optional, the Google Cloud project ID in which to start a job. If set to None or missing, the default project_id from the Google Cloud connection is used. :param location: Job location. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param poll_sleep: The time in seconds to sleep between polling Google Cloud Platform for the dataflow job status while the job is in the JOB_STATE_RUNNING state. @@ -1173,7 +1120,6 @@ def __init__( project_id: str | None = None, location: str = DEFAULT_DATAFLOW_LOCATION, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, poll_sleep: int = 10, drain_pipeline: bool = False, cancel_timeout: int | None = 10 * 60, @@ -1203,11 +1149,6 @@ def __init__( self.project_id = project_id self.location = location self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.poll_sleep = poll_sleep self.drain_pipeline = drain_pipeline self.cancel_timeout = cancel_timeout @@ -1221,7 +1162,6 @@ def execute(self, context: Context): self.beam_hook = BeamHook(runner=BeamRunnerType.DataflowRunner) self.dataflow_hook = DataflowHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, poll_sleep=self.poll_sleep, impersonation_chain=None, drain_pipeline=self.drain_pipeline, @@ -1249,7 +1189,7 @@ def set_current_job_id(job_id): with ExitStack() as exit_stack: if self.py_file.lower().startswith("gs://"): - gcs_hook = GCSHook(self.gcp_conn_id, self.delegate_to) + gcs_hook = GCSHook(self.gcp_conn_id) tmp_gcs_file = exit_stack.enter_context(gcs_hook.provide_file(object_url=self.py_file)) self.py_file = tmp_gcs_file.name @@ -1303,9 +1243,6 @@ class DataflowStopJobOperator(GoogleCloudBaseOperator): If set to None or missing, the default project_id from the Google Cloud connection is used. :param location: Optional, Job location. If set to None or missing, "us-central1" will be used. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param poll_sleep: The time in seconds to sleep between polling Google Cloud Platform for the dataflow job status to confirm it's stopped. :param impersonation_chain: Optional service account to impersonate using short-term @@ -1328,7 +1265,6 @@ def __init__( project_id: str | None = None, location: str = DEFAULT_DATAFLOW_LOCATION, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, poll_sleep: int = 10, impersonation_chain: str | Sequence[str] | None = None, stop_timeout: int | None = 10 * 60, @@ -1343,11 +1279,6 @@ def __init__( self.project_id = project_id self.location = location self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain self.hook: DataflowHook | None = None self.drain_pipeline = drain_pipeline @@ -1355,7 +1286,6 @@ def __init__( def execute(self, context: Context) -> None: self.dataflow_hook = DataflowHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, poll_sleep=self.poll_sleep, impersonation_chain=self.impersonation_chain, cancel_timeout=self.stop_timeout, diff --git a/airflow/providers/google/cloud/operators/dataform.py b/airflow/providers/google/cloud/operators/dataform.py index 8e43cf7f8ac3c..184b7f0fc24bf 100644 --- a/airflow/providers/google/cloud/operators/dataform.py +++ b/airflow/providers/google/cloud/operators/dataform.py @@ -16,7 +16,6 @@ # under the License. from __future__ import annotations -import warnings from typing import TYPE_CHECKING, Sequence from airflow.providers.google.cloud.links.dataform import ( @@ -56,8 +55,6 @@ class DataformCreateCompilationResultOperator(GoogleCloudBaseOperator): :param timeout: The timeout for this request. :param metadata: Strings which should be sent along with the request as metadata. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate, if any. For this to work, the service accountmaking the - request must have domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -78,7 +75,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, *args, **kwargs, @@ -92,17 +88,11 @@ def __init__( self.timeout = timeout self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context): hook = DataformHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) result = hook.create_compilation_result( @@ -129,8 +119,6 @@ class DataformGetCompilationResultOperator(GoogleCloudBaseOperator): :param timeout: The timeout for this request. :param metadata: Strings which should be sent along with the request as metadata. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate, if any. For this to work, the service accountmaking the - request must have domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -141,7 +129,7 @@ class DataformGetCompilationResultOperator(GoogleCloudBaseOperator): account from the list granting this role to the originating account (templated). """ - template_fields = ("repository_id", "compilation_result_id", "delegate_to", "impersonation_chain") + template_fields = ("repository_id", "compilation_result_id", "impersonation_chain") def __init__( self, @@ -153,7 +141,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, *args, **kwargs, @@ -167,17 +154,11 @@ def __init__( self.timeout = timeout self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context): hook = DataformHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) result = hook.get_compilation_result( @@ -204,8 +185,6 @@ class DataformCreateWorkflowInvocationOperator(GoogleCloudBaseOperator): :param timeout: The timeout for this request. :param metadata: Strings which should be sent along with the request as metadata. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate, if any. For this to work, the service accountmaking the - request must have domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -215,12 +194,12 @@ class DataformCreateWorkflowInvocationOperator(GoogleCloudBaseOperator): Service Account Token Creator IAM role to the directly preceding identity, with first account from the list granting this role to the originating account (templated). :param asynchronous: Flag to return workflow_invocation_id from the Dataform API. - This is useful for submitting long running workflows and + This is useful for submitting long-running workflows and waiting on them asynchronously using the DataformWorkflowInvocationStateSensor :param wait_time: Number of seconds between checks """ - template_fields = ("workflow_invocation", "delegate_to", "impersonation_chain") + template_fields = ("workflow_invocation", "impersonation_chain") operator_extra_links = (DataformWorkflowInvocationLink(),) def __init__( @@ -233,7 +212,6 @@ def __init__( timeout: int | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, asynchronous: bool = False, wait_time: int = 10, @@ -249,11 +227,6 @@ def __init__( self.timeout = timeout self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain self.asynchronous = asynchronous self.wait_time = wait_time @@ -261,7 +234,6 @@ def __init__( def execute(self, context: Context): hook = DataformHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) result = hook.create_workflow_invocation( @@ -306,8 +278,6 @@ class DataformGetWorkflowInvocationOperator(GoogleCloudBaseOperator): :param timeout: The timeout for this request. :param metadata: Strings which should be sent along with the request as metadata. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate, if any. For this to work, the service accountmaking the - request must have domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -318,7 +288,7 @@ class DataformGetWorkflowInvocationOperator(GoogleCloudBaseOperator): account from the list granting this role to the originating account (templated). """ - template_fields = ("repository_id", "workflow_invocation_id", "delegate_to", "impersonation_chain") + template_fields = ("repository_id", "workflow_invocation_id", "impersonation_chain") operator_extra_links = (DataformWorkflowInvocationLink(),) def __init__( @@ -331,7 +301,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, *args, **kwargs, @@ -345,17 +314,11 @@ def __init__( self.timeout = timeout self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context): hook = DataformHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) result = hook.get_workflow_invocation( @@ -382,8 +345,6 @@ class DataformCancelWorkflowInvocationOperator(GoogleCloudBaseOperator): :param timeout: The timeout for this request. :param metadata: Strings which should be sent along with the request as metadata. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate, if any. For this to work, the service accountmaking the - request must have domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -394,7 +355,7 @@ class DataformCancelWorkflowInvocationOperator(GoogleCloudBaseOperator): account from the list granting this role to the originating account (templated). """ - template_fields = ("repository_id", "workflow_invocation_id", "delegate_to", "impersonation_chain") + template_fields = ("repository_id", "workflow_invocation_id", "impersonation_chain") operator_extra_links = (DataformWorkflowInvocationLink(),) def __init__( @@ -407,7 +368,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, *args, **kwargs, @@ -421,17 +381,11 @@ def __init__( self.timeout = timeout self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context): hook = DataformHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) hook.cancel_workflow_invocation( @@ -456,8 +410,6 @@ class DataformCreateRepositoryOperator(GoogleCloudBaseOperator): :param timeout: The timeout for this request. :param metadata: Strings which should be sent along with the request as metadata. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate, if any. For this to work, the service accountmaking the - request must have domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -472,7 +424,6 @@ class DataformCreateRepositoryOperator(GoogleCloudBaseOperator): template_fields = ( "project_id", "repository_id", - "delegate_to", "impersonation_chain", ) @@ -485,7 +436,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, *args, **kwargs, @@ -501,17 +451,11 @@ def __init__( self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context) -> dict: hook = DataformHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) @@ -546,8 +490,6 @@ class DataformDeleteRepositoryOperator(GoogleCloudBaseOperator): :param timeout: The timeout for this request. :param metadata: Strings which should be sent along with the request as metadata. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate, if any. For this to work, the service accountmaking the - request must have domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -561,7 +503,6 @@ class DataformDeleteRepositoryOperator(GoogleCloudBaseOperator): template_fields = ( "project_id", "repository_id", - "delegate_to", "impersonation_chain", ) @@ -575,7 +516,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, *args, **kwargs, @@ -592,17 +532,11 @@ def __init__( self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context) -> None: hook = DataformHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) @@ -629,8 +563,6 @@ class DataformCreateWorkspaceOperator(GoogleCloudBaseOperator): :param timeout: The timeout for this request. :param metadata: Strings which should be sent along with the request as metadata. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate, if any. For this to work, the service accountmaking the - request must have domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -645,7 +577,6 @@ class DataformCreateWorkspaceOperator(GoogleCloudBaseOperator): template_fields = ( "project_id", "repository_id", - "delegate_to", "impersonation_chain", ) @@ -659,7 +590,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, *args, **kwargs, @@ -676,17 +606,11 @@ def __init__( self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context) -> dict: hook = DataformHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) @@ -724,8 +648,6 @@ class DataformDeleteWorkspaceOperator(GoogleCloudBaseOperator): :param timeout: The timeout for this request. :param metadata: Strings which should be sent along with the request as metadata. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate, if any. For this to work, the service accountmaking the - request must have domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -740,7 +662,6 @@ class DataformDeleteWorkspaceOperator(GoogleCloudBaseOperator): "project_id", "repository_id", "workspace_id", - "delegate_to", "impersonation_chain", ) @@ -754,7 +675,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, *args, **kwargs, @@ -771,17 +691,11 @@ def __init__( self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context) -> None: hook = DataformHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) @@ -810,8 +724,6 @@ class DataformWriteFileOperator(GoogleCloudBaseOperator): :param timeout: The timeout for this request. :param metadata: Strings which should be sent along with the request as metadata. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate, if any. For this to work, the service accountmaking the - request must have domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -826,7 +738,6 @@ class DataformWriteFileOperator(GoogleCloudBaseOperator): "project_id", "repository_id", "workspace_id", - "delegate_to", "impersonation_chain", ) @@ -842,7 +753,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, *args, **kwargs, @@ -861,17 +771,11 @@ def __init__( self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context) -> dict: hook = DataformHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) write_file_response = hook.write_file( @@ -901,8 +805,6 @@ class DataformMakeDirectoryOperator(GoogleCloudBaseOperator): :param timeout: The timeout for this request. :param metadata: Strings which should be sent along with the request as metadata. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate, if any. For this to work, the service accountmaking the - request must have domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -917,7 +819,6 @@ class DataformMakeDirectoryOperator(GoogleCloudBaseOperator): "project_id", "repository_id", "workspace_id", - "delegate_to", "impersonation_chain", ) @@ -932,7 +833,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, *args, **kwargs, @@ -950,17 +850,11 @@ def __init__( self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context) -> dict: hook = DataformHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) @@ -991,8 +885,6 @@ class DataformRemoveFileOperator(GoogleCloudBaseOperator): :param timeout: The timeout for this request. :param metadata: Strings which should be sent along with the request as metadata. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate, if any. For this to work, the service accountmaking the - request must have domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -1007,7 +899,6 @@ class DataformRemoveFileOperator(GoogleCloudBaseOperator): "project_id", "repository_id", "workspace_id", - "delegate_to", "impersonation_chain", ) @@ -1022,7 +913,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, *args, **kwargs, @@ -1040,17 +930,11 @@ def __init__( self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context) -> None: hook = DataformHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) @@ -1079,8 +963,6 @@ class DataformRemoveDirectoryOperator(GoogleCloudBaseOperator): :param timeout: The timeout for this request. :param metadata: Strings which should be sent along with the request as metadata. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate, if any. For this to work, the service accountmaking the - request must have domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -1095,7 +977,6 @@ class DataformRemoveDirectoryOperator(GoogleCloudBaseOperator): "project_id", "repository_id", "workspace_id", - "delegate_to", "impersonation_chain", ) @@ -1110,7 +991,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, *args, **kwargs, @@ -1128,17 +1008,11 @@ def __init__( self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context) -> None: hook = DataformHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) @@ -1166,8 +1040,6 @@ class DataformInstallNpmPackagesOperator(GoogleCloudBaseOperator): :param timeout: The timeout for this request. :param metadata: Strings which should be sent along with the request as metadata. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate, if any. For this to work, the service accountmaking the - request must have domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -1182,7 +1054,6 @@ class DataformInstallNpmPackagesOperator(GoogleCloudBaseOperator): "project_id", "repository_id", "workspace_id", - "delegate_to", "impersonation_chain", ) @@ -1196,7 +1067,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, *args, **kwargs, @@ -1213,17 +1083,11 @@ def __init__( self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context) -> dict: hook = DataformHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) diff --git a/airflow/providers/google/cloud/operators/datafusion.py b/airflow/providers/google/cloud/operators/datafusion.py index b16adc94af705..635c8edd9586d 100644 --- a/airflow/providers/google/cloud/operators/datafusion.py +++ b/airflow/providers/google/cloud/operators/datafusion.py @@ -17,7 +17,6 @@ """This module contains Google DataFusion operators.""" from __future__ import annotations -import warnings from time import sleep from typing import TYPE_CHECKING, Any, Sequence @@ -62,9 +61,6 @@ class CloudDataFusionRestartInstanceOperator(GoogleCloudBaseOperator): :param project_id: The ID of the Google Cloud project that the instance belongs to. :param api_version: The version of the api that will be requested for example 'v3'. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -89,7 +85,6 @@ def __init__( project_id: str | None = None, api_version: str = "v1beta1", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -99,17 +94,11 @@ def __init__( self.project_id = project_id self.api_version = api_version self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context) -> None: hook = DataFusionHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, api_version=self.api_version, impersonation_chain=self.impersonation_chain, ) @@ -145,9 +134,6 @@ class CloudDataFusionDeleteInstanceOperator(GoogleCloudBaseOperator): :param project_id: The ID of the Google Cloud project that the instance belongs to. :param api_version: The version of the api that will be requested for example 'v3'. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -171,7 +157,6 @@ def __init__( project_id: str | None = None, api_version: str = "v1beta1", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -181,17 +166,11 @@ def __init__( self.project_id = project_id self.api_version = api_version self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context) -> None: hook = DataFusionHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, api_version=self.api_version, impersonation_chain=self.impersonation_chain, ) @@ -220,9 +199,6 @@ class CloudDataFusionCreateInstanceOperator(GoogleCloudBaseOperator): :param project_id: The ID of the Google Cloud project that the instance belongs to. :param api_version: The version of the api that will be requested for example 'v3'. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -249,7 +225,6 @@ def __init__( project_id: str | None = None, api_version: str = "v1beta1", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -260,17 +235,11 @@ def __init__( self.project_id = project_id self.api_version = api_version self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context) -> dict: hook = DataFusionHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, api_version=self.api_version, impersonation_chain=self.impersonation_chain, ) @@ -332,9 +301,6 @@ class CloudDataFusionUpdateInstanceOperator(GoogleCloudBaseOperator): :param project_id: The ID of the Google Cloud project that the instance belongs to. :param api_version: The version of the api that will be requested for example 'v3'. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -362,7 +328,6 @@ def __init__( project_id: str | None = None, api_version: str = "v1beta1", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -374,17 +339,11 @@ def __init__( self.project_id = project_id self.api_version = api_version self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context) -> None: hook = DataFusionHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, api_version=self.api_version, impersonation_chain=self.impersonation_chain, ) @@ -422,9 +381,6 @@ class CloudDataFusionGetInstanceOperator(GoogleCloudBaseOperator): :param project_id: The ID of the Google Cloud project that the instance belongs to. :param api_version: The version of the api that will be requested for example 'v3'. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -449,7 +405,6 @@ def __init__( project_id: str | None = None, api_version: str = "v1beta1", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -459,17 +414,11 @@ def __init__( self.project_id = project_id self.api_version = api_version self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context) -> dict: hook = DataFusionHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, api_version=self.api_version, impersonation_chain=self.impersonation_chain, ) @@ -509,9 +458,6 @@ class CloudDataFusionCreatePipelineOperator(GoogleCloudBaseOperator): can create a namespace. :param api_version: The version of the api that will be requested for example 'v3'. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -541,7 +487,6 @@ def __init__( project_id: str | None = None, api_version: str = "v1beta1", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -554,17 +499,11 @@ def __init__( self.project_id = project_id self.api_version = api_version self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context) -> None: hook = DataFusionHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, api_version=self.api_version, impersonation_chain=self.impersonation_chain, ) @@ -607,9 +546,6 @@ class CloudDataFusionDeletePipelineOperator(GoogleCloudBaseOperator): can create a namespace. :param api_version: The version of the api that will be requested for example 'v3'. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -638,7 +574,6 @@ def __init__( project_id: str | None = None, api_version: str = "v1beta1", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -651,17 +586,11 @@ def __init__( self.project_id = project_id self.api_version = api_version self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context) -> None: hook = DataFusionHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, api_version=self.api_version, impersonation_chain=self.impersonation_chain, ) @@ -699,9 +628,6 @@ class CloudDataFusionListPipelinesOperator(GoogleCloudBaseOperator): can create a namespace. :param api_version: The version of the api that will be requested for example 'v3'. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -731,7 +657,6 @@ def __init__( project_id: str | None = None, api_version: str = "v1beta1", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -744,17 +669,11 @@ def __init__( self.project_id = project_id self.api_version = api_version self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context) -> dict: hook = DataFusionHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, api_version=self.api_version, impersonation_chain=self.impersonation_chain, ) @@ -799,9 +718,6 @@ class CloudDataFusionStartPipelineOperator(GoogleCloudBaseOperator): can create a namespace. :param api_version: The version of the api that will be requested for example 'v3'. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -841,7 +757,6 @@ def __init__( project_id: str | None = None, api_version: str = "v1beta1", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, asynchronous=False, deferrable=False, @@ -857,11 +772,6 @@ def __init__( self.project_id = project_id self.api_version = api_version self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain self.asynchronous = asynchronous self.pipeline_timeout = pipeline_timeout @@ -876,7 +786,6 @@ def __init__( def execute(self, context: Context) -> str: hook = DataFusionHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, api_version=self.api_version, impersonation_chain=self.impersonation_chain, ) @@ -917,7 +826,6 @@ def execute(self, context: Context) -> str: poll_interval=self.poll_interval, gcp_conn_id=self.gcp_conn_id, impersonation_chain=self.impersonation_chain, - delegate_to=self.delegate_to, ), method_name="execute_complete", ) @@ -969,9 +877,6 @@ class CloudDataFusionStopPipelineOperator(GoogleCloudBaseOperator): can create a namespace. :param api_version: The version of the api that will be requested for example 'v3'. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -999,7 +904,6 @@ def __init__( project_id: str | None = None, api_version: str = "v1beta1", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -1011,17 +915,11 @@ def __init__( self.project_id = project_id self.api_version = api_version self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context) -> None: hook = DataFusionHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, api_version=self.api_version, impersonation_chain=self.impersonation_chain, ) diff --git a/airflow/providers/google/cloud/operators/dataplex.py b/airflow/providers/google/cloud/operators/dataplex.py index 2fc329397e2d8..7b50523e54f8b 100644 --- a/airflow/providers/google/cloud/operators/dataplex.py +++ b/airflow/providers/google/cloud/operators/dataplex.py @@ -18,7 +18,6 @@ from __future__ import annotations -import warnings from time import sleep from typing import TYPE_CHECKING, Any, Sequence @@ -57,8 +56,6 @@ class DataplexCreateTaskOperator(GoogleCloudBaseOperator): Note that if `retry` is specified, the timeout applies to each individual attempt. :param metadata: Additional metadata that is provided to the method. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate, if any. For this to work, the service accountmaking the - request must have domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -77,7 +74,6 @@ class DataplexCreateTaskOperator(GoogleCloudBaseOperator): "dataplex_task_id", "body", "validate_only", - "delegate_to", "impersonation_chain", ) template_fields_renderers = {"body": "json"} @@ -96,7 +92,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, asynchronous: bool = False, *args, @@ -114,18 +109,12 @@ def __init__( self.timeout = timeout self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain self.asynchronous = asynchronous def execute(self, context: Context) -> dict: hook = DataplexHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, api_version=self.api_version, impersonation_chain=self.impersonation_chain, ) @@ -189,8 +178,6 @@ class DataplexDeleteTaskOperator(GoogleCloudBaseOperator): Note that if `retry` is specified, the timeout applies to each individual attempt. :param metadata: Additional metadata that is provided to the method. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate, if any. For this to work, the service accountmaking the - request must have domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -201,7 +188,7 @@ class DataplexDeleteTaskOperator(GoogleCloudBaseOperator): account from the list granting this role to the originating account (templated). """ - template_fields = ("project_id", "dataplex_task_id", "delegate_to", "impersonation_chain") + template_fields = ("project_id", "dataplex_task_id", "impersonation_chain") def __init__( self, @@ -214,7 +201,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, *args, **kwargs, @@ -229,17 +215,11 @@ def __init__( self.timeout = timeout self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context) -> None: hook = DataplexHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, api_version=self.api_version, impersonation_chain=self.impersonation_chain, ) @@ -280,8 +260,6 @@ class DataplexListTasksOperator(GoogleCloudBaseOperator): Note that if `retry` is specified, the timeout applies to each individual attempt. :param metadata: Additional metadata that is provided to the method. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate, if any. For this to work, the service accountmaking the - request must have domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -298,7 +276,6 @@ class DataplexListTasksOperator(GoogleCloudBaseOperator): "page_token", "filter", "order_by", - "delegate_to", "impersonation_chain", ) operator_extra_links = (DataplexTasksLink(),) @@ -317,7 +294,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, *args, **kwargs, @@ -335,17 +311,11 @@ def __init__( self.timeout = timeout self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context) -> list[dict]: hook = DataplexHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, api_version=self.api_version, impersonation_chain=self.impersonation_chain, ) @@ -382,8 +352,6 @@ class DataplexGetTaskOperator(GoogleCloudBaseOperator): Note that if `retry` is specified, the timeout applies to each individual attempt. :param metadata: Additional metadata that is provided to the method. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate, if any. For this to work, the service accountmaking the - request must have domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -394,7 +362,7 @@ class DataplexGetTaskOperator(GoogleCloudBaseOperator): account from the list granting this role to the originating account (templated). """ - template_fields = ("project_id", "dataplex_task_id", "delegate_to", "impersonation_chain") + template_fields = ("project_id", "dataplex_task_id", "impersonation_chain") operator_extra_links = (DataplexTaskLink(),) def __init__( @@ -408,7 +376,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, *args, **kwargs, @@ -423,17 +390,11 @@ def __init__( self.timeout = timeout self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context) -> dict: hook = DataplexHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, api_version=self.api_version, impersonation_chain=self.impersonation_chain, ) @@ -470,8 +431,6 @@ class DataplexCreateLakeOperator(GoogleCloudBaseOperator): Note that if `retry` is specified, the timeout applies to each individual attempt. :param metadata: Additional metadata that is provided to the method. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate, if any. For this to work, the service account making the - request must have domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -490,7 +449,6 @@ class DataplexCreateLakeOperator(GoogleCloudBaseOperator): "lake_id", "body", "validate_only", - "delegate_to", "impersonation_chain", ) template_fields_renderers = {"body": "json"} @@ -508,7 +466,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, asynchronous: bool = False, *args, @@ -525,18 +482,12 @@ def __init__( self.timeout = timeout self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain self.asynchronous = asynchronous def execute(self, context: Context) -> dict: hook = DataplexHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, api_version=self.api_version, impersonation_chain=self.impersonation_chain, ) @@ -599,8 +550,6 @@ class DataplexDeleteLakeOperator(GoogleCloudBaseOperator): Note that if `retry` is specified, the timeout applies to each individual attempt. :param metadata: Additional metadata that is provided to the method. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate, if any. For this to work, the service account making the - request must have domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -611,7 +560,7 @@ class DataplexDeleteLakeOperator(GoogleCloudBaseOperator): account from the list granting this role to the originating account (templated). """ - template_fields = ("project_id", "lake_id", "delegate_to", "impersonation_chain") + template_fields = ("project_id", "lake_id", "impersonation_chain") operator_extra_links = (DataplexLakeLink(),) def __init__( @@ -624,7 +573,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, *args, **kwargs, @@ -639,18 +587,12 @@ def __init__( self.timeout = timeout self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context) -> None: hook = DataplexHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, api_version=self.api_version, impersonation_chain=self.impersonation_chain, ) diff --git a/airflow/providers/google/cloud/operators/dataproc.py b/airflow/providers/google/cloud/operators/dataproc.py index 6ca7b4d9dcdb7..219396793c8a9 100644 --- a/airflow/providers/google/cloud/operators/dataproc.py +++ b/airflow/providers/google/cloud/operators/dataproc.py @@ -929,9 +929,6 @@ class DataprocJobBaseOperator(GoogleCloudBaseOperator): :param dataproc_jars: HCFS URIs of jar files to add to the CLASSPATH of the Hive server and Hadoop MapReduce (MR) tasks. Can contain Hive SerDes and UDFs. (templated) :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param labels: The labels to associate with this job. Label keys must contain 1 to 63 characters, and must conform to RFC 1035. Label values may be empty, but, if present, must contain 1 to 63 characters, and must conform to RFC 1035. No more than 32 labels can be associated with a job. @@ -977,7 +974,6 @@ def __init__( dataproc_properties: dict | None = None, dataproc_jars: list[str] | None = None, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, labels: dict | None = None, job_error_states: set[str] | None = None, impersonation_chain: str | Sequence[str] | None = None, @@ -990,11 +986,6 @@ def __init__( if deferrable and polling_interval_seconds <= 0: raise ValueError("Invalid value for polling_interval_seconds. Expected value greater than 0") self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.labels = labels self.job_name = job_name self.cluster_name = cluster_name @@ -1062,7 +1053,6 @@ def execute(self, context: Context): job_id=job_id, project_id=self.project_id, region=self.region, - delegate_to=self.delegate_to, gcp_conn_id=self.gcp_conn_id, impersonation_chain=self.impersonation_chain, polling_interval_seconds=self.polling_interval_seconds, diff --git a/airflow/providers/google/cloud/operators/datastore.py b/airflow/providers/google/cloud/operators/datastore.py index 91aa8be977973..bcd651f11f5cf 100644 --- a/airflow/providers/google/cloud/operators/datastore.py +++ b/airflow/providers/google/cloud/operators/datastore.py @@ -18,7 +18,6 @@ """This module contains Google Datastore operators.""" from __future__ import annotations -import warnings from typing import TYPE_CHECKING, Any, Sequence from airflow.exceptions import AirflowException @@ -46,15 +45,12 @@ class CloudDatastoreExportEntitiesOperator(GoogleCloudBaseOperator): .. seealso:: https://cloud.google.com/datastore/docs/export-import-entities - :param bucket: name of the cloud storage bucket to backup data + :param bucket: name of the cloud storage bucket to back up data :param namespace: optional namespace path in the specified Cloud Storage bucket - to backup data. If this namespace does not exist in GCS, it will be created. + to back up data. If this namespace does not exist in GCS, it will be created. :param datastore_conn_id: the name of the Datastore connection id to use :param cloud_storage_conn_id: the name of the cloud storage connection id to force-write backup - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param entity_filter: description of what data from the project is included in the export, refer to https://cloud.google.com/datastore/docs/reference/rest/Shared.Types/EntityFilter @@ -89,7 +85,6 @@ def __init__( namespace: str | None = None, datastore_conn_id: str = "google_cloud_default", cloud_storage_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, entity_filter: dict | None = None, labels: dict | None = None, polling_interval_in_seconds: int = 10, @@ -101,11 +96,6 @@ def __init__( super().__init__(**kwargs) self.datastore_conn_id = datastore_conn_id self.cloud_storage_conn_id = cloud_storage_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.bucket = bucket self.namespace = namespace self.entity_filter = entity_filter @@ -126,7 +116,6 @@ def execute(self, context: Context) -> dict: ds_hook = DatastoreHook( gcp_conn_id=self.datastore_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) result = ds_hook.export_to_storage_bucket( @@ -172,9 +161,6 @@ class CloudDatastoreImportEntitiesOperator(GoogleCloudBaseOperator): https://cloud.google.com/datastore/docs/reference/rest/Shared.Types/EntityFilter :param labels: client-assigned labels for cloud storage :param datastore_conn_id: the name of the connection id to use - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param polling_interval_in_seconds: number of seconds to wait before polling for execution status again :param impersonation_chain: Optional service account to impersonate using short-term @@ -206,7 +192,6 @@ def __init__( entity_filter: dict | None = None, labels: dict | None = None, datastore_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, polling_interval_in_seconds: float = 10, project_id: str | None = None, impersonation_chain: str | Sequence[str] | None = None, @@ -214,11 +199,6 @@ def __init__( ) -> None: super().__init__(**kwargs) self.datastore_conn_id = datastore_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.bucket = bucket self.file = file self.namespace = namespace @@ -232,7 +212,6 @@ def execute(self, context: Context): self.log.info("Importing data from Cloud Storage bucket %s", self.bucket) ds_hook = DatastoreHook( self.datastore_conn_id, - self.delegate_to, impersonation_chain=self.impersonation_chain, ) result = ds_hook.import_from_storage_bucket( @@ -267,9 +246,6 @@ class CloudDatastoreAllocateIdsOperator(GoogleCloudBaseOperator): :param partial_keys: a list of partial keys. :param project_id: Google Cloud project ID against which to make the request. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token @@ -292,7 +268,6 @@ def __init__( *, partial_keys: list, project_id: str | None = None, - delegate_to: str | None = None, gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, **kwargs, @@ -302,11 +277,6 @@ def __init__( self.partial_keys = partial_keys self.gcp_conn_id = gcp_conn_id self.project_id = project_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context) -> list: @@ -335,9 +305,6 @@ class CloudDatastoreBeginTransactionOperator(GoogleCloudBaseOperator): :param transaction_options: Options for a new transaction. :param project_id: Google Cloud project ID against which to make the request. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token @@ -359,7 +326,6 @@ def __init__( *, transaction_options: dict[str, Any], project_id: str | None = None, - delegate_to: str | None = None, gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, **kwargs, @@ -369,11 +335,6 @@ def __init__( self.transaction_options = transaction_options self.gcp_conn_id = gcp_conn_id self.project_id = project_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context) -> str: @@ -401,9 +362,6 @@ class CloudDatastoreCommitOperator(GoogleCloudBaseOperator): :param body: the body of the commit request. :param project_id: Google Cloud project ID against which to make the request. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token @@ -426,7 +384,6 @@ def __init__( *, body: dict[str, Any], project_id: str | None = None, - delegate_to: str | None = None, gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, **kwargs, @@ -436,11 +393,6 @@ def __init__( self.body = body self.gcp_conn_id = gcp_conn_id self.project_id = project_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context) -> dict: @@ -469,9 +421,6 @@ class CloudDatastoreRollbackOperator(GoogleCloudBaseOperator): :param transaction: the transaction to roll back. :param project_id: Google Cloud project ID against which to make the request. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token @@ -493,7 +442,6 @@ def __init__( *, transaction: str, project_id: str | None = None, - delegate_to: str | None = None, gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, **kwargs, @@ -503,11 +451,6 @@ def __init__( self.transaction = transaction self.gcp_conn_id = gcp_conn_id self.project_id = project_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context) -> None: @@ -534,9 +477,6 @@ class CloudDatastoreRunQueryOperator(GoogleCloudBaseOperator): :param body: the body of the query request. :param project_id: Google Cloud project ID against which to make the request. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token @@ -558,7 +498,6 @@ def __init__( *, body: dict[str, Any], project_id: str | None = None, - delegate_to: str | None = None, gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, **kwargs, @@ -568,11 +507,6 @@ def __init__( self.body = body self.gcp_conn_id = gcp_conn_id self.project_id = project_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context) -> dict: @@ -599,9 +533,6 @@ class CloudDatastoreGetOperationOperator(GoogleCloudBaseOperator): https://cloud.google.com/datastore/docs/reference/data/rest/v1/projects.operations/get :param name: the name of the operation resource. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token @@ -622,7 +553,6 @@ def __init__( self, *, name: str, - delegate_to: str | None = None, gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, **kwargs, @@ -631,11 +561,6 @@ def __init__( self.name = name self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context): @@ -659,9 +584,6 @@ class CloudDatastoreDeleteOperationOperator(GoogleCloudBaseOperator): https://cloud.google.com/datastore/docs/reference/data/rest/v1/projects.operations/delete :param name: the name of the operation resource. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token @@ -682,7 +604,6 @@ def __init__( self, *, name: str, - delegate_to: str | None = None, gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, **kwargs, @@ -691,11 +612,6 @@ def __init__( self.name = name self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context) -> None: diff --git a/airflow/providers/google/cloud/operators/gcs.py b/airflow/providers/google/cloud/operators/gcs.py index 026367c951ae8..e2936c0933cc1 100644 --- a/airflow/providers/google/cloud/operators/gcs.py +++ b/airflow/providers/google/cloud/operators/gcs.py @@ -21,7 +21,6 @@ import datetime import subprocess import sys -import warnings from pathlib import Path from tempfile import NamedTemporaryFile, TemporaryDirectory from typing import TYPE_CHECKING, Sequence @@ -74,9 +73,6 @@ class GCSCreateBucketOperator(GoogleCloudBaseOperator): :param project_id: The ID of the Google Cloud Project. (templated) :param labels: User-provided labels, in key/value pairs. :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -122,7 +118,6 @@ def __init__( project_id: str | None = None, labels: dict | None = None, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -134,17 +129,11 @@ def __init__( self.project_id = project_id self.labels = labels self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context) -> None: hook = GCSHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) StorageLink.persist( @@ -177,12 +166,9 @@ class GCSListObjectsOperator(GoogleCloudBaseOperator): :param prefix: Prefix string which filters objects whose name begin with this prefix. (templated) :param delimiter: The delimiter by which you want to filter the objects. (templated) - For e.g to lists the CSV files from in a directory in GCS you would use + For example, to lists the CSV files from in a directory in GCS you would use delimiter='.csv'. :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -223,7 +209,6 @@ def __init__( prefix: str | None = None, delimiter: str | None = None, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -232,18 +217,12 @@ def __init__( self.prefix = prefix self.delimiter = delimiter self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context) -> list: hook = GCSHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) @@ -276,9 +255,6 @@ class GCSDeleteObjectsOperator(GoogleCloudBaseOperator): :param prefix: Prefix of objects to delete. All objects matching this prefix in the bucket will be deleted. :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -303,7 +279,6 @@ def __init__( objects: list[str] | None = None, prefix: str | None = None, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -312,11 +287,6 @@ def __init__( self.objects = objects self.prefix = prefix self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain if objects is None and prefix is None: @@ -330,7 +300,6 @@ def __init__( def execute(self, context: Context) -> None: hook = GCSHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) @@ -944,9 +913,6 @@ class GCSSynchronizeBucketsOperator(GoogleCloudBaseOperator): This option can delete data quickly if you specify the wrong source/destination combination. :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -966,7 +932,6 @@ class GCSSynchronizeBucketsOperator(GoogleCloudBaseOperator): "delete_extra_files", "allow_overwrite", "gcp_conn_id", - "delegate_to", "impersonation_chain", ) operator_extra_links = (StorageLink(),) @@ -982,7 +947,6 @@ def __init__( delete_extra_files: bool = False, allow_overwrite: bool = False, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -995,17 +959,11 @@ def __init__( self.delete_extra_files = delete_extra_files self.allow_overwrite = allow_overwrite self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context) -> None: hook = GCSHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) StorageLink.persist( diff --git a/airflow/providers/google/cloud/operators/mlengine.py b/airflow/providers/google/cloud/operators/mlengine.py index 64a242ed759c1..7e35b7b1f76b0 100644 --- a/airflow/providers/google/cloud/operators/mlengine.py +++ b/airflow/providers/google/cloud/operators/mlengine.py @@ -145,9 +145,6 @@ class MLEngineStartBatchPredictionJobOperator(GoogleCloudBaseOperator): (templated) :param gcp_conn_id: The connection ID used for connection to Google Cloud Platform. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param labels: a dictionary containing labels for the job; passed to BigQuery :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token @@ -190,7 +187,6 @@ def __init__( signature_name: str | None = None, project_id: str | None = None, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, labels: dict[str, str] | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, @@ -210,11 +206,6 @@ def __init__( self._runtime_version = runtime_version self._signature_name = signature_name self._gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self._delegate_to = delegate_to self._labels = labels self._impersonation_chain = impersonation_chain @@ -274,9 +265,7 @@ def execute(self, context: Context): if self._signature_name: prediction_request["predictionInput"]["signatureName"] = self._signature_name - hook = MLEngineHook( - self._gcp_conn_id, self._delegate_to, impersonation_chain=self._impersonation_chain - ) + hook = MLEngineHook(gcp_conn_id=self._gcp_conn_id, impersonation_chain=self._impersonation_chain) # Helper method to check if the existing job's prediction input is the # same as the request we get here. @@ -316,9 +305,6 @@ class MLEngineManageModelOperator(GoogleCloudBaseOperator): If set to None or missing, the default project_id from the Google Cloud connection is used. (templated) :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -342,7 +328,6 @@ def __init__( operation: str = "create", project_id: str | None = None, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -359,17 +344,11 @@ def __init__( self._model = model self._operation = operation self._gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self._delegate_to = delegate_to self._impersonation_chain = impersonation_chain def execute(self, context: Context): hook = MLEngineHook( gcp_conn_id=self._gcp_conn_id, - delegate_to=self._delegate_to, impersonation_chain=self._impersonation_chain, ) if self._operation == "create": @@ -395,9 +374,6 @@ class MLEngineCreateModelOperator(GoogleCloudBaseOperator): If set to None or missing, the default project_id from the Google Cloud connection is used. (templated) :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -421,7 +397,6 @@ def __init__( model: dict, project_id: str | None = None, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -429,17 +404,11 @@ def __init__( self._project_id = project_id self._model = model self._gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self._delegate_to = delegate_to self._impersonation_chain = impersonation_chain def execute(self, context: Context): hook = MLEngineHook( gcp_conn_id=self._gcp_conn_id, - delegate_to=self._delegate_to, impersonation_chain=self._impersonation_chain, ) @@ -470,9 +439,6 @@ class MLEngineGetModelOperator(GoogleCloudBaseOperator): If set to None or missing, the default project_id from the Google Cloud connection is used. (templated) :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -496,7 +462,6 @@ def __init__( model_name: str, project_id: str | None = None, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -504,17 +469,11 @@ def __init__( self._project_id = project_id self._model_name = model_name self._gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self._delegate_to = delegate_to self._impersonation_chain = impersonation_chain def execute(self, context: Context): hook = MLEngineHook( gcp_conn_id=self._gcp_conn_id, - delegate_to=self._delegate_to, impersonation_chain=self._impersonation_chain, ) project_id = self._project_id or hook.project_id @@ -547,9 +506,6 @@ class MLEngineDeleteModelOperator(GoogleCloudBaseOperator): If set to None or missing, the default project_id from the Google Cloud connection is used. (templated) :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -574,7 +530,6 @@ def __init__( delete_contents: bool = False, project_id: str | None = None, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -583,17 +538,11 @@ def __init__( self._model_name = model_name self._delete_contents = delete_contents self._gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self._delegate_to = delegate_to self._impersonation_chain = impersonation_chain def execute(self, context: Context): hook = MLEngineHook( gcp_conn_id=self._gcp_conn_id, - delegate_to=self._delegate_to, impersonation_chain=self._impersonation_chain, ) @@ -653,9 +602,6 @@ class MLEngineManageVersionOperator(GoogleCloudBaseOperator): If set to None or missing, the default project_id from the Google Cloud connection is used. (templated) :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -683,7 +629,6 @@ def __init__( operation: str = "create", project_id: str | None = None, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -694,11 +639,6 @@ def __init__( self._version = version or {} self._operation = operation self._gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self._delegate_to = delegate_to self._impersonation_chain = impersonation_chain warnings.warn( @@ -714,7 +654,6 @@ def execute(self, context: Context): hook = MLEngineHook( gcp_conn_id=self._gcp_conn_id, - delegate_to=self._delegate_to, impersonation_chain=self._impersonation_chain, ) @@ -755,9 +694,6 @@ class MLEngineCreateVersionOperator(GoogleCloudBaseOperator): If set to None or missing, the default project_id from the Google Cloud connection is used. (templated) :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -783,7 +719,6 @@ def __init__( version: dict, project_id: str | None = None, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -793,11 +728,6 @@ def __init__( self._model_name = model_name self._version = version self._gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self._delegate_to = delegate_to self._impersonation_chain = impersonation_chain self._validate_inputs() @@ -811,7 +741,6 @@ def _validate_inputs(self): def execute(self, context: Context): hook = MLEngineHook( gcp_conn_id=self._gcp_conn_id, - delegate_to=self._delegate_to, impersonation_chain=self._impersonation_chain, ) @@ -847,9 +776,6 @@ class MLEngineSetDefaultVersionOperator(GoogleCloudBaseOperator): If set to None or missing, the default project_id from the Google Cloud connection is used. (templated) :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -875,7 +801,6 @@ def __init__( version_name: str, project_id: str | None = None, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -885,11 +810,6 @@ def __init__( self._model_name = model_name self._version_name = version_name self._gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self._delegate_to = delegate_to self._impersonation_chain = impersonation_chain self._validate_inputs() @@ -903,7 +823,6 @@ def _validate_inputs(self): def execute(self, context: Context): hook = MLEngineHook( gcp_conn_id=self._gcp_conn_id, - delegate_to=self._delegate_to, impersonation_chain=self._impersonation_chain, ) @@ -938,9 +857,6 @@ class MLEngineListVersionsOperator(GoogleCloudBaseOperator): :param project_id: The Google Cloud project name to which MLEngine model belongs. If set to None or missing, the default project_id from the Google Cloud connection is used. (templated) - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -964,7 +880,6 @@ def __init__( model_name: str, project_id: str | None = None, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -973,11 +888,6 @@ def __init__( self._project_id = project_id self._model_name = model_name self._gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self._delegate_to = delegate_to self._impersonation_chain = impersonation_chain self._validate_inputs() @@ -988,7 +898,6 @@ def _validate_inputs(self): def execute(self, context: Context): hook = MLEngineHook( gcp_conn_id=self._gcp_conn_id, - delegate_to=self._delegate_to, impersonation_chain=self._impersonation_chain, ) @@ -1024,9 +933,6 @@ class MLEngineDeleteVersionOperator(GoogleCloudBaseOperator): :param project_id: The Google Cloud project name to which MLEngine model belongs. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -1052,7 +958,6 @@ def __init__( version_name: str, project_id: str | None = None, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -1062,11 +967,6 @@ def __init__( self._model_name = model_name self._version_name = version_name self._gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self._delegate_to = delegate_to self._impersonation_chain = impersonation_chain self._validate_inputs() @@ -1080,7 +980,6 @@ def _validate_inputs(self): def execute(self, context: Context): hook = MLEngineHook( gcp_conn_id=self._gcp_conn_id, - delegate_to=self._delegate_to, impersonation_chain=self._impersonation_chain, ) @@ -1139,9 +1038,6 @@ class MLEngineStartTrainingJobOperator(GoogleCloudBaseOperator): If set to None or missing, the Google-managed Cloud ML Engine service account will be used. :param project_id: The Google Cloud project name within which MLEngine training job should run. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param mode: Can be one of 'DRY_RUN'/'CLOUD'. In 'DRY_RUN' mode, no real training job will be launched, but the MLEngine training job request will be printed out. In 'CLOUD' mode, a real MLEngine training job @@ -1198,7 +1094,6 @@ def __init__( job_dir: str | None = None, service_account: str | None = None, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, mode: str = "PRODUCTION", labels: dict[str, str] | None = None, impersonation_chain: str | Sequence[str] | None = None, @@ -1222,11 +1117,6 @@ def __init__( self._job_dir = job_dir self._service_account = service_account self._gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self._delegate_to = delegate_to self._mode = mode self._labels = labels self._hyperparameters = hyperparameters @@ -1316,7 +1206,6 @@ def execute(self, context: Context): hook = MLEngineHook( gcp_conn_id=self._gcp_conn_id, - delegate_to=self._delegate_to, impersonation_chain=self._impersonation_chain, ) self.hook = hook @@ -1363,7 +1252,6 @@ def execute(self, context: Context): labels=self._labels, gcp_conn_id=self._gcp_conn_id, impersonation_chain=self._impersonation_chain, - delegate_to=self._delegate_to, ), method_name="execute_complete", ) @@ -1446,9 +1334,6 @@ class MLEngineTrainingCancelJobOperator(GoogleCloudBaseOperator): If set to None or missing, the default project_id from the Google Cloud connection is used. (templated) :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -1472,7 +1357,6 @@ def __init__( job_id: str, project_id: str | None = None, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -1480,11 +1364,6 @@ def __init__( self._project_id = project_id self._job_id = job_id self._gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self._delegate_to = delegate_to self._impersonation_chain = impersonation_chain if not self._project_id: @@ -1494,7 +1373,6 @@ def execute(self, context: Context): hook = MLEngineHook( gcp_conn_id=self._gcp_conn_id, - delegate_to=self._delegate_to, impersonation_chain=self._impersonation_chain, ) diff --git a/airflow/providers/google/cloud/operators/pubsub.py b/airflow/providers/google/cloud/operators/pubsub.py index 420934b2463dd..b8dfe62961f53 100644 --- a/airflow/providers/google/cloud/operators/pubsub.py +++ b/airflow/providers/google/cloud/operators/pubsub.py @@ -24,7 +24,6 @@ """ from __future__ import annotations -import warnings from typing import TYPE_CHECKING, Any, Callable, Sequence from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault @@ -85,9 +84,6 @@ class PubSubCreateTopicOperator(GoogleCloudBaseOperator): ``{topic}``. (templated) :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param labels: Client-assigned labels; see https://cloud.google.com/pubsub/docs/labels :param message_storage_policy: Policy constraining the set @@ -130,7 +126,6 @@ def __init__( project_id: str | None = None, fail_if_exists: bool = False, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, labels: dict[str, str] | None = None, message_storage_policy: dict | MessageStoragePolicy = None, kms_key_name: str | None = None, @@ -146,11 +141,6 @@ def __init__( self.topic = topic self.fail_if_exists = fail_if_exists self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.labels = labels self.message_storage_policy = message_storage_policy self.kms_key_name = kms_key_name @@ -162,7 +152,6 @@ def __init__( def execute(self, context: Context) -> None: hook = PubSubHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) @@ -256,9 +245,6 @@ class PubSubCreateSubscriptionOperator(GoogleCloudBaseOperator): acknowledge each message pulled from the subscription :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param push_config: If push delivery is used with this subscription, this field is used to configure it. An empty ``pushConfig`` signifies that the subscriber will pull and ack messages using API methods. @@ -332,7 +318,6 @@ def __init__( ack_deadline_secs: int = 10, fail_if_exists: bool = False, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, push_config: dict | PushConfig | None = None, retain_acked_messages: bool | None = None, message_retention_duration: dict | Duration | None = None, @@ -356,11 +341,6 @@ def __init__( self.ack_deadline_secs = ack_deadline_secs self.fail_if_exists = fail_if_exists self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.push_config = push_config self.retain_acked_messages = retain_acked_messages self.message_retention_duration = message_retention_duration @@ -378,7 +358,6 @@ def __init__( def execute(self, context: Context) -> str: hook = PubSubHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) @@ -446,9 +425,6 @@ class PubSubDeleteTopicOperator(GoogleCloudBaseOperator): the task :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param retry: (Optional) A retry object used to retry requests. If None is specified, requests will not be retried. :param timeout: (Optional) The amount of time, in seconds, to wait for the request @@ -479,7 +455,6 @@ def __init__( project_id: str | None = None, fail_if_not_exists: bool = False, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, retry: Retry | _MethodDefault = DEFAULT, timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), @@ -491,11 +466,6 @@ def __init__( self.topic = topic self.fail_if_not_exists = fail_if_not_exists self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.retry = retry self.timeout = timeout self.metadata = metadata @@ -504,7 +474,6 @@ def __init__( def execute(self, context: Context) -> None: hook = PubSubHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) @@ -554,9 +523,6 @@ class PubSubDeleteSubscriptionOperator(GoogleCloudBaseOperator): fail the task :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param retry: (Optional) A retry object used to retry requests. If None is specified, requests will not be retried. :param timeout: (Optional) The amount of time, in seconds, to wait for the request @@ -587,7 +553,6 @@ def __init__( project_id: str | None = None, fail_if_not_exists: bool = False, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, retry: Retry | _MethodDefault = DEFAULT, timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), @@ -599,11 +564,6 @@ def __init__( self.subscription = subscription self.fail_if_not_exists = fail_if_not_exists self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.retry = retry self.timeout = timeout self.metadata = metadata @@ -612,7 +572,6 @@ def __init__( def execute(self, context: Context) -> None: hook = PubSubHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) @@ -672,9 +631,6 @@ class PubSubPublishMessageOperator(GoogleCloudBaseOperator): https://cloud.google.com/pubsub/docs/reference/rest/v1/PubsubMessage :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -700,7 +656,6 @@ def __init__( messages: list, project_id: str | None = None, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -709,17 +664,11 @@ def __init__( self.topic = topic self.messages = messages self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context) -> None: hook = PubSubHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) @@ -757,9 +706,6 @@ class PubSubPullOperator(GoogleCloudBaseOperator): immediately rather than by any downstream tasks :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param messages_callback: (Optional) Callback to process received messages. It's return value will be saved to XCom. If you are pulling large messages, you probably want to provide a custom callback. @@ -790,17 +736,11 @@ def __init__( ack_messages: bool = False, messages_callback: Callable[[list[ReceivedMessage], Context], Any] | None = None, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: super().__init__(**kwargs) self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.project_id = project_id self.subscription = subscription self.max_messages = max_messages @@ -811,7 +751,6 @@ def __init__( def execute(self, context: Context) -> list: hook = PubSubHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) diff --git a/airflow/providers/google/cloud/operators/stackdriver.py b/airflow/providers/google/cloud/operators/stackdriver.py index 8eaea721bd583..33a082355e0db 100644 --- a/airflow/providers/google/cloud/operators/stackdriver.py +++ b/airflow/providers/google/cloud/operators/stackdriver.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import warnings from typing import TYPE_CHECKING, Sequence from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault @@ -71,9 +70,6 @@ class StackdriverListAlertPoliciesOperator(GoogleCloudBaseOperator): :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform. :param project_id: The project to fetch alerts from. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -103,7 +99,6 @@ def __init__( metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", project_id: str | None = None, - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -117,11 +112,6 @@ def __init__( self.metadata = metadata self.gcp_conn_id = gcp_conn_id self.project_id = project_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain self.hook: StackdriverHook | None = None @@ -137,7 +127,6 @@ def execute(self, context: Context): if self.hook is None: self.hook = StackdriverHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) @@ -180,9 +169,6 @@ class StackdriverEnableAlertPoliciesOperator(GoogleCloudBaseOperator): :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform. :param project_id: The project in which alert needs to be enabled. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -209,18 +195,12 @@ def __init__( metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", project_id: str | None = None, - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: super().__init__(**kwargs) self.gcp_conn_id = gcp_conn_id self.project_id = project_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.filter_ = filter_ self.retry = retry self.timeout = timeout @@ -233,7 +213,6 @@ def execute(self, context: Context): if self.hook is None: self.hook = StackdriverHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) self.hook.enable_alert_policies( @@ -272,9 +251,6 @@ class StackdriverDisableAlertPoliciesOperator(GoogleCloudBaseOperator): :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform. :param project_id: The project in which alert needs to be disabled. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -301,18 +277,12 @@ def __init__( metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", project_id: str | None = None, - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: super().__init__(**kwargs) self.gcp_conn_id = gcp_conn_id self.project_id = project_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.filter_ = filter_ self.retry = retry self.timeout = timeout @@ -325,7 +295,6 @@ def execute(self, context: Context): if self.hook is None: self.hook = StackdriverHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) self.hook.disable_alert_policies( @@ -364,9 +333,6 @@ class StackdriverUpsertAlertOperator(GoogleCloudBaseOperator): :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform. :param project_id: The project in which alert needs to be created/updated. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -395,7 +361,6 @@ def __init__( metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", project_id: str | None = None, - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -406,11 +371,6 @@ def __init__( self.metadata = metadata self.gcp_conn_id = gcp_conn_id self.project_id = project_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain self.hook: StackdriverHook | None = None @@ -419,7 +379,6 @@ def execute(self, context: Context): if self.hook is None: self.hook = StackdriverHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) self.hook.upsert_alert( @@ -455,9 +414,6 @@ class StackdriverDeleteAlertOperator(GoogleCloudBaseOperator): :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform. :param project_id: The project from which alert needs to be deleted. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -484,7 +440,6 @@ def __init__( metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", project_id: str | None = None, - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -495,11 +450,6 @@ def __init__( self.metadata = metadata self.gcp_conn_id = gcp_conn_id self.project_id = project_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain self.hook: StackdriverHook | None = None @@ -508,7 +458,6 @@ def execute(self, context: Context): if self.hook is None: self.hook = StackdriverHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) self.hook.delete_alert_policy( @@ -555,9 +504,6 @@ class StackdriverListNotificationChannelsOperator(GoogleCloudBaseOperator): :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform. :param project_id: The project to fetch notification channels from. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -588,7 +534,6 @@ def __init__( metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", project_id: str | None = None, - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -602,11 +547,6 @@ def __init__( self.metadata = metadata self.gcp_conn_id = gcp_conn_id self.project_id = project_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain self.hook: StackdriverHook | None = None @@ -622,7 +562,6 @@ def execute(self, context: Context): if self.hook is None: self.hook = StackdriverHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) channels = self.hook.list_notification_channels( @@ -664,9 +603,6 @@ class StackdriverEnableNotificationChannelsOperator(GoogleCloudBaseOperator): :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform. :param project_id: The location used for the operation. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -694,7 +630,6 @@ def __init__( metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", project_id: str | None = None, - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -705,11 +640,6 @@ def __init__( self.metadata = metadata self.gcp_conn_id = gcp_conn_id self.project_id = project_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain self.hook: StackdriverHook | None = None @@ -720,7 +650,6 @@ def execute(self, context: Context): if self.hook is None: self.hook = StackdriverHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) self.hook.enable_notification_channels( @@ -758,9 +687,6 @@ class StackdriverDisableNotificationChannelsOperator(GoogleCloudBaseOperator): :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform. :param project_id: The project in which notification channels needs to be enabled. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -788,7 +714,6 @@ def __init__( metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", project_id: str | None = None, - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -799,11 +724,6 @@ def __init__( self.metadata = metadata self.gcp_conn_id = gcp_conn_id self.project_id = project_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain self.hook: StackdriverHook | None = None @@ -814,7 +734,6 @@ def execute(self, context: Context): if self.hook is None: self.hook = StackdriverHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) self.hook.disable_notification_channels( @@ -853,9 +772,6 @@ class StackdriverUpsertNotificationChannelOperator(GoogleCloudBaseOperator): :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform. :param project_id: The project in which notification channels needs to be created/updated. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -884,7 +800,6 @@ def __init__( metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", project_id: str | None = None, - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -895,11 +810,6 @@ def __init__( self.metadata = metadata self.gcp_conn_id = gcp_conn_id self.project_id = project_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain self.hook: StackdriverHook | None = None @@ -910,7 +820,6 @@ def execute(self, context: Context): if self.hook is None: self.hook = StackdriverHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) self.hook.upsert_channel( @@ -946,9 +855,6 @@ class StackdriverDeleteNotificationChannelOperator(GoogleCloudBaseOperator): :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform. :param project_id: The project from which notification channel needs to be deleted. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -975,7 +881,6 @@ def __init__( metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", project_id: str | None = None, - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -986,11 +891,6 @@ def __init__( self.metadata = metadata self.gcp_conn_id = gcp_conn_id self.project_id = project_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain self.hook: StackdriverHook | None = None @@ -999,7 +899,6 @@ def execute(self, context: Context): if self.hook is None: self.hook = StackdriverHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) self.hook.delete_notification_channel( diff --git a/airflow/providers/google/cloud/operators/vertex_ai/auto_ml.py b/airflow/providers/google/cloud/operators/vertex_ai/auto_ml.py index 0e80ffe5f6c24..f933cfc376563 100644 --- a/airflow/providers/google/cloud/operators/vertex_ai/auto_ml.py +++ b/airflow/providers/google/cloud/operators/vertex_ai/auto_ml.py @@ -18,7 +18,6 @@ """This module contains Google Vertex AI operators.""" from __future__ import annotations -import warnings from typing import TYPE_CHECKING, Sequence from google.api_core.exceptions import NotFound @@ -59,7 +58,6 @@ def __init__( model_labels: dict[str, str] | None = None, sync: bool = True, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -78,11 +76,6 @@ def __init__( self.sync = sync # END Run param self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain self.hook: AutoMLHook | None = None @@ -165,7 +158,6 @@ def __init__( def execute(self, context: Context): self.hook = AutoMLHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) model, training_id = self.hook.create_auto_ml_forecasting_training_job( @@ -262,7 +254,6 @@ def __init__( def execute(self, context: Context): self.hook = AutoMLHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) model, training_id = self.hook.create_auto_ml_image_training_job( @@ -359,7 +350,6 @@ def __init__( def execute(self, context: Context): self.hook = AutoMLHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) model, training_id = self.hook.create_auto_ml_tabular_training_job( @@ -444,7 +434,6 @@ def __init__( def execute(self, context: Context): self.hook = AutoMLHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) model, training_id = self.hook.create_auto_ml_text_training_job( @@ -510,7 +499,6 @@ def __init__( def execute(self, context: Context): self.hook = AutoMLHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) model, training_id = self.hook.create_auto_ml_video_training_job( @@ -560,7 +548,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -572,17 +559,11 @@ def __init__( self.timeout = timeout self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context): hook = AutoMLHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) try: @@ -628,7 +609,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -643,17 +623,11 @@ def __init__( self.timeout = timeout self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context): hook = AutoMLHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) results = hook.list_training_pipelines( diff --git a/airflow/providers/google/cloud/operators/vertex_ai/batch_prediction_job.py b/airflow/providers/google/cloud/operators/vertex_ai/batch_prediction_job.py index 9477e339eb6e4..5c123227751e0 100644 --- a/airflow/providers/google/cloud/operators/vertex_ai/batch_prediction_job.py +++ b/airflow/providers/google/cloud/operators/vertex_ai/batch_prediction_job.py @@ -26,7 +26,6 @@ """ from __future__ import annotations -import warnings from typing import TYPE_CHECKING, Sequence from google.api_core.exceptions import NotFound @@ -144,9 +143,6 @@ class CreateBatchPredictionJobOperator(GoogleCloudBaseOperator): :param timeout: The timeout for this request. :param metadata: Strings which should be sent along with the request as metadata. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -186,7 +182,6 @@ def __init__( encryption_spec_key_name: str | None = None, sync: bool = True, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -214,11 +209,6 @@ def __init__( self.encryption_spec_key_name = encryption_spec_key_name self.sync = sync self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain self.hook: BatchPredictionJobHook | None = None @@ -226,7 +216,6 @@ def execute(self, context: Context): self.log.info("Creating Batch prediction job") self.hook = BatchPredictionJobHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) result = self.hook.create_batch_prediction_job( @@ -284,9 +273,6 @@ class DeleteBatchPredictionJobOperator(GoogleCloudBaseOperator): :param timeout: The timeout for this request. :param metadata: Strings which should be sent along with the request as metadata. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -309,7 +295,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -321,17 +306,11 @@ def __init__( self.timeout = timeout self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context): hook = BatchPredictionJobHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) @@ -362,9 +341,6 @@ class GetBatchPredictionJobOperator(GoogleCloudBaseOperator): :param timeout: The timeout for this request. :param metadata: Strings which should be sent along with the request as metadata. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -388,7 +364,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -400,17 +375,11 @@ def __init__( self.timeout = timeout self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context): hook = BatchPredictionJobHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) @@ -456,9 +425,6 @@ class ListBatchPredictionJobsOperator(GoogleCloudBaseOperator): :param timeout: The timeout for this request. :param metadata: Strings which should be sent along with the request as metadata. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -485,7 +451,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -500,17 +465,11 @@ def __init__( self.timeout = timeout self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context): hook = BatchPredictionJobHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) results = hook.list_batch_prediction_jobs( diff --git a/airflow/providers/google/cloud/operators/vertex_ai/custom_job.py b/airflow/providers/google/cloud/operators/vertex_ai/custom_job.py index b3cfe7dcece54..38c3a2ea69445 100644 --- a/airflow/providers/google/cloud/operators/vertex_ai/custom_job.py +++ b/airflow/providers/google/cloud/operators/vertex_ai/custom_job.py @@ -18,7 +18,6 @@ """This module contains Google Vertex AI operators.""" from __future__ import annotations -import warnings from typing import TYPE_CHECKING, Sequence from google.api_core.exceptions import NotFound @@ -93,7 +92,6 @@ def __init__( tensorboard: str | None = None, sync=True, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -148,11 +146,6 @@ def __init__( self.sync = sync # END Run param self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain @@ -402,9 +395,6 @@ class CreateCustomContainerTrainingJobOperator(CustomTrainingJobBaseOperator): will be executed in concurrent Future and any downstream object will be immediately returned and synced when the Future has completed. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -435,7 +425,6 @@ def __init__( def execute(self, context: Context): self.hook = CustomJobHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) model, training_id, custom_job_id = self.hook.create_custom_container_training_job( @@ -754,9 +743,6 @@ class CreateCustomPythonPackageTrainingJobOperator(CustomTrainingJobBaseOperator will be executed in concurrent Future and any downstream object will be immediately returned and synced when the Future has completed. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -788,7 +774,6 @@ def __init__( def execute(self, context: Context): self.hook = CustomJobHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) model, training_id, custom_job_id = self.hook.create_custom_python_package_training_job( @@ -1108,9 +1093,6 @@ class CreateCustomTrainingJobOperator(CustomTrainingJobBaseOperator): will be executed in concurrent Future and any downstream object will be immediately returned and synced when the Future has completed. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -1144,7 +1126,6 @@ def __init__( def execute(self, context: Context): self.hook = CustomJobHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) model, training_id, custom_job_id = self.hook.create_custom_training_job( @@ -1229,9 +1210,6 @@ class DeleteCustomTrainingJobOperator(GoogleCloudBaseOperator): :param timeout: The timeout for this request. :param metadata: Strings which should be sent along with the request as metadata. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -1255,7 +1233,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -1268,17 +1245,11 @@ def __init__( self.timeout = timeout self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context): hook = CustomJobHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) try: @@ -1342,9 +1313,6 @@ class ListCustomTrainingJobOperator(GoogleCloudBaseOperator): :param timeout: The timeout for this request. :param metadata: Strings which should be sent along with the request as metadata. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -1377,7 +1345,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -1392,17 +1359,11 @@ def __init__( self.timeout = timeout self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context): hook = CustomJobHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) results = hook.list_training_pipelines( diff --git a/airflow/providers/google/cloud/operators/vertex_ai/dataset.py b/airflow/providers/google/cloud/operators/vertex_ai/dataset.py index f3cf59873ef14..222cc659727fa 100644 --- a/airflow/providers/google/cloud/operators/vertex_ai/dataset.py +++ b/airflow/providers/google/cloud/operators/vertex_ai/dataset.py @@ -18,7 +18,6 @@ """This module contains Google Vertex AI operators.""" from __future__ import annotations -import warnings from typing import TYPE_CHECKING, Sequence from google.api_core.exceptions import NotFound @@ -47,9 +46,6 @@ class CreateDatasetOperator(GoogleCloudBaseOperator): :param timeout: The timeout for this request. :param metadata: Strings which should be sent along with the request as metadata. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -73,7 +69,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -85,17 +80,11 @@ def __init__( self.timeout = timeout self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context): hook = DatasetHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) @@ -130,9 +119,6 @@ class GetDatasetOperator(GoogleCloudBaseOperator): :param timeout: The timeout for this request. :param metadata: Strings which should be sent along with the request as metadata. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -157,7 +143,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -170,17 +155,11 @@ def __init__( self.timeout = timeout self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context): hook = DatasetHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) @@ -213,9 +192,6 @@ class DeleteDatasetOperator(GoogleCloudBaseOperator): :param timeout: The timeout for this request. :param metadata: Strings which should be sent along with the request as metadata. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -238,7 +214,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -250,17 +225,11 @@ def __init__( self.timeout = timeout self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context): hook = DatasetHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) @@ -292,9 +261,6 @@ class ExportDataOperator(GoogleCloudBaseOperator): :param timeout: The timeout for this request. :param metadata: Strings which should be sent along with the request as metadata. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -318,7 +284,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -331,17 +296,11 @@ def __init__( self.timeout = timeout self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context): hook = DatasetHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) @@ -372,9 +331,6 @@ class ImportDataOperator(GoogleCloudBaseOperator): :param timeout: The timeout for this request. :param metadata: Strings which should be sent along with the request as metadata. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -398,7 +354,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -411,17 +366,11 @@ def __init__( self.timeout = timeout self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context): hook = DatasetHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) @@ -455,9 +404,6 @@ class ListDatasetsOperator(GoogleCloudBaseOperator): :param timeout: The timeout for this request. :param metadata: Strings which should be sent along with the request as metadata. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -485,7 +431,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -501,17 +446,11 @@ def __init__( self.timeout = timeout self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context): hook = DatasetHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) results = hook.list_datasets( @@ -543,9 +482,6 @@ class UpdateDatasetOperator(GoogleCloudBaseOperator): :param timeout: The timeout for this request. :param metadata: Strings which should be sent along with the request as metadata. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -570,7 +506,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -584,17 +519,11 @@ def __init__( self.timeout = timeout self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context): hook = DatasetHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) self.log.info("Updating dataset: %s", self.dataset_id) diff --git a/airflow/providers/google/cloud/operators/vertex_ai/endpoint_service.py b/airflow/providers/google/cloud/operators/vertex_ai/endpoint_service.py index d48f874f1305e..30d097d673751 100644 --- a/airflow/providers/google/cloud/operators/vertex_ai/endpoint_service.py +++ b/airflow/providers/google/cloud/operators/vertex_ai/endpoint_service.py @@ -28,7 +28,6 @@ """ from __future__ import annotations -import warnings from typing import TYPE_CHECKING, Sequence from google.api_core.exceptions import NotFound @@ -62,9 +61,6 @@ class CreateEndpointOperator(GoogleCloudBaseOperator): :param timeout: The timeout for this request. :param metadata: Strings which should be sent along with the request as metadata. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -89,7 +85,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -102,17 +97,11 @@ def __init__( self.timeout = timeout self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context): hook = EndpointServiceHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) @@ -148,9 +137,6 @@ class DeleteEndpointOperator(GoogleCloudBaseOperator): :param timeout: The timeout for this request. :param metadata: Strings which should be sent along with the request as metadata. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -173,7 +159,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -185,17 +170,11 @@ def __init__( self.timeout = timeout self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context): hook = EndpointServiceHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) @@ -242,9 +221,6 @@ class DeployModelOperator(GoogleCloudBaseOperator): :param timeout: The timeout for this request. :param metadata: Strings which should be sent along with the request as metadata. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -270,7 +246,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -284,17 +259,11 @@ def __init__( self.timeout = timeout self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context): hook = EndpointServiceHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) @@ -331,9 +300,6 @@ class GetEndpointOperator(GoogleCloudBaseOperator): :param timeout: The timeout for this request. :param metadata: Strings which should be sent along with the request as metadata. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -357,7 +323,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -369,17 +334,11 @@ def __init__( self.timeout = timeout self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context): hook = EndpointServiceHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) @@ -432,9 +391,6 @@ class ListEndpointsOperator(GoogleCloudBaseOperator): :param timeout: The timeout for this request. :param metadata: Strings which should be sent along with the request as metadata. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -462,7 +418,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -478,17 +433,11 @@ def __init__( self.timeout = timeout self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context): hook = EndpointServiceHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) results = hook.list_endpoints( @@ -527,9 +476,6 @@ class UndeployModelOperator(GoogleCloudBaseOperator): :param timeout: The timeout for this request. :param metadata: Strings which should be sent along with the request as metadata. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -554,7 +500,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -568,17 +513,11 @@ def __init__( self.timeout = timeout self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context): hook = EndpointServiceHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) @@ -611,9 +550,6 @@ class UpdateEndpointOperator(GoogleCloudBaseOperator): :param timeout: The timeout for this request. :param metadata: Strings which should be sent along with the request as metadata. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -639,7 +575,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -653,17 +588,11 @@ def __init__( self.timeout = timeout self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context): hook = EndpointServiceHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) self.log.info("Updating endpoint: %s", self.endpoint_id) diff --git a/airflow/providers/google/cloud/operators/vertex_ai/hyperparameter_tuning_job.py b/airflow/providers/google/cloud/operators/vertex_ai/hyperparameter_tuning_job.py index df1e6790aacf6..42be6fbadd20b 100644 --- a/airflow/providers/google/cloud/operators/vertex_ai/hyperparameter_tuning_job.py +++ b/airflow/providers/google/cloud/operators/vertex_ai/hyperparameter_tuning_job.py @@ -27,7 +27,6 @@ """ from __future__ import annotations -import warnings from typing import TYPE_CHECKING, Sequence from google.api_core.exceptions import NotFound @@ -134,9 +133,6 @@ class CreateHyperparameterTuningJobOperator(GoogleCloudBaseOperator): :param sync: Whether to execute this method synchronously. If False, this method will unblock and it will be executed in a concurrent Future. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -186,7 +182,6 @@ def __init__( sync: bool = True, # END: run param gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -218,11 +213,6 @@ def __init__( self.tensorboard = tensorboard self.sync = sync self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain self.hook: HyperparameterTuningJobHook | None = None @@ -230,7 +220,6 @@ def execute(self, context: Context): self.log.info("Creating Hyperparameter Tuning job") self.hook = HyperparameterTuningJobHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) result = self.hook.create_hyperparameter_tuning_job( @@ -294,9 +283,6 @@ class GetHyperparameterTuningJobOperator(GoogleCloudBaseOperator): :param timeout: The timeout for this request. :param metadata: Strings which should be sent along with the request as metadata. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -320,7 +306,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -332,17 +317,11 @@ def __init__( self.timeout = timeout self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context): hook = HyperparameterTuningJobHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) @@ -392,7 +371,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -404,17 +382,11 @@ def __init__( self.timeout = timeout self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context): hook = HyperparameterTuningJobHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) try: @@ -479,7 +451,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -494,17 +465,11 @@ def __init__( self.timeout = timeout self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context): hook = HyperparameterTuningJobHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) results = hook.list_hyperparameter_tuning_jobs( diff --git a/airflow/providers/google/cloud/operators/vertex_ai/model_service.py b/airflow/providers/google/cloud/operators/vertex_ai/model_service.py index 10eb092f7c0b5..e21601ee90da1 100644 --- a/airflow/providers/google/cloud/operators/vertex_ai/model_service.py +++ b/airflow/providers/google/cloud/operators/vertex_ai/model_service.py @@ -24,7 +24,6 @@ """ from __future__ import annotations -import warnings from typing import TYPE_CHECKING, Sequence from google.api_core.exceptions import NotFound @@ -55,9 +54,6 @@ class DeleteModelOperator(GoogleCloudBaseOperator): :param timeout: The timeout for this request. :param metadata: Strings which should be sent along with the request as metadata. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -80,7 +76,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -92,17 +87,11 @@ def __init__( self.timeout = timeout self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context): hook = ModelServiceHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) @@ -134,9 +123,6 @@ class ExportModelOperator(GoogleCloudBaseOperator): :param timeout: The timeout for this request. :param metadata: Strings which should be sent along with the request as metadata. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -161,7 +147,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -174,17 +159,11 @@ def __init__( self.timeout = timeout self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context): hook = ModelServiceHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) @@ -234,9 +213,6 @@ class ListModelsOperator(GoogleCloudBaseOperator): :param timeout: The timeout for this request. :param metadata: Strings which should be sent along with the request as metadata. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -264,7 +240,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -280,17 +255,11 @@ def __init__( self.timeout = timeout self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context): hook = ModelServiceHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) results = hook.list_models( @@ -320,9 +289,6 @@ class UploadModelOperator(GoogleCloudBaseOperator): :param timeout: The timeout for this request. :param metadata: Strings which should be sent along with the request as metadata. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -346,7 +312,6 @@ def __init__( timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -358,17 +323,11 @@ def __init__( self.timeout = timeout self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context): hook = ModelServiceHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) self.log.info("Upload model") diff --git a/airflow/providers/google/cloud/sensors/bigquery.py b/airflow/providers/google/cloud/sensors/bigquery.py index 5bd6212c75b85..fba30a5fca99d 100644 --- a/airflow/providers/google/cloud/sensors/bigquery.py +++ b/airflow/providers/google/cloud/sensors/bigquery.py @@ -45,9 +45,6 @@ class BigQueryTableExistenceSensor(BaseSensorOperator): storage bucket. :param table_id: The name of the table to check the existence of. :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -73,7 +70,6 @@ def __init__( dataset_id: str, table_id: str, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, deferrable: bool = False, **kwargs, @@ -97,11 +93,6 @@ def __init__( self.dataset_id = dataset_id self.table_id = table_id self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain self.deferrable = deferrable @@ -111,7 +102,6 @@ def poke(self, context: Context) -> bool: self.log.info("Sensor checks existence of table: %s", table_uri) hook = BigQueryHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) return hook.table_exists( @@ -129,7 +119,6 @@ def execute(self, context: Context) -> None: poll_interval=self.poke_interval, gcp_conn_id=self.gcp_conn_id, hook_params={ - "delegate_to": self.delegate_to, "impersonation_chain": self.impersonation_chain, }, ), @@ -163,9 +152,6 @@ class BigQueryTablePartitionExistenceSensor(BaseSensorOperator): :param table_id: The name of the table to check the existence of. :param partition_id: The name of the partition to check the existence of. :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud. - :param delegate_to: The account to impersonate, if any. - For this to work, the service account making the request must - have domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -193,7 +179,6 @@ def __init__( table_id: str, partition_id: str, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, deferrable: bool = False, **kwargs, @@ -207,11 +192,6 @@ def __init__( self.table_id = table_id self.partition_id = partition_id self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain self.deferrable = deferrable @@ -221,7 +201,6 @@ def poke(self, context: Context) -> bool: self.log.info('Sensor checks existence of partition: "%s" in table: %s', self.partition_id, table_uri) hook = BigQueryHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) return hook.table_partition_exists( @@ -283,9 +262,6 @@ class BigQueryTableExistenceAsyncSensor(BigQueryTableExistenceSensor): :param gcp_conn_id: The connection ID used to connect to Google Cloud. :param bigquery_conn_id: (Deprecated) The connection ID used to connect to Google Cloud. This parameter has been deprecated. You should pass the gcp_conn_id parameter instead. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. diff --git a/airflow/providers/google/cloud/sensors/cloud_composer.py b/airflow/providers/google/cloud/sensors/cloud_composer.py index c566587e02671..f454d255a1693 100644 --- a/airflow/providers/google/cloud/sensors/cloud_composer.py +++ b/airflow/providers/google/cloud/sensors/cloud_composer.py @@ -19,7 +19,6 @@ from __future__ import annotations -import warnings from typing import TYPE_CHECKING, Any, Sequence from airflow.exceptions import AirflowException @@ -38,8 +37,6 @@ class CloudComposerEnvironmentSensor(BaseSensorOperator): :param region: Required. The ID of the Google Cloud region that the service belongs to. :param operation_name: The name of the operation resource :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate, if any. For this to work, the service account making the - request must have domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -58,7 +55,6 @@ def __init__( region: str, operation_name: str, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, pooling_period_seconds: int = 30, **kwargs, @@ -69,11 +65,6 @@ def __init__( self.operation_name = operation_name self.pooling_period_seconds = pooling_period_seconds self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context) -> None: @@ -85,7 +76,6 @@ def execute(self, context: Context) -> None: operation_name=self.operation_name, gcp_conn_id=self.gcp_conn_id, impersonation_chain=self.impersonation_chain, - delegate_to=self.delegate_to, pooling_period_seconds=self.pooling_period_seconds, ), method_name="execute_complete", diff --git a/airflow/providers/google/cloud/sensors/dataflow.py b/airflow/providers/google/cloud/sensors/dataflow.py index fc55f1c881b0b..187b4c00070ab 100644 --- a/airflow/providers/google/cloud/sensors/dataflow.py +++ b/airflow/providers/google/cloud/sensors/dataflow.py @@ -18,7 +18,6 @@ """This module contains a Google Cloud Dataflow sensor.""" from __future__ import annotations -import warnings from typing import TYPE_CHECKING, Callable, Sequence from airflow.exceptions import AirflowException @@ -50,10 +49,6 @@ class DataflowJobStatusSensor(BaseSensorOperator): :param location: The location of the Dataflow job (for example europe-west1). See: https://cloud.google.com/dataflow/docs/concepts/regional-endpoints :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. See: - https://developers.google.com/identity/protocols/oauth2/service-account#delegatingauthority :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -74,7 +69,6 @@ def __init__( project_id: str | None = None, location: str = DEFAULT_DATAFLOW_LOCATION, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -86,11 +80,6 @@ def __init__( self.project_id = project_id self.location = location self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain self.hook: DataflowHook | None = None @@ -102,7 +91,6 @@ def poke(self, context: Context) -> bool: ) self.hook = DataflowHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) @@ -142,9 +130,6 @@ class DataflowJobMetricsSensor(BaseSensorOperator): :param location: The location of the Dataflow job (for example europe-west1). See: https://cloud.google.com/dataflow/docs/concepts/regional-endpoints :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -166,7 +151,6 @@ def __init__( project_id: str | None = None, location: str = DEFAULT_DATAFLOW_LOCATION, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -177,18 +161,12 @@ def __init__( self.fail_on_terminal_state = fail_on_terminal_state self.location = location self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain self.hook: DataflowHook | None = None def poke(self, context: Context) -> bool: self.hook = DataflowHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) @@ -231,9 +209,6 @@ class DataflowJobMessagesSensor(BaseSensorOperator): If set to None or missing, the default project_id from the Google Cloud connection is used. :param location: Job location. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -255,7 +230,6 @@ def __init__( project_id: str | None = None, location: str = DEFAULT_DATAFLOW_LOCATION, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -266,18 +240,12 @@ def __init__( self.fail_on_terminal_state = fail_on_terminal_state self.location = location self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain self.hook: DataflowHook | None = None def poke(self, context: Context) -> bool: self.hook = DataflowHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) @@ -320,9 +288,6 @@ class DataflowJobAutoScalingEventsSensor(BaseSensorOperator): If set to None or missing, the default project_id from the Google Cloud connection is used. :param location: Job location. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -344,7 +309,6 @@ def __init__( project_id: str | None = None, location: str = DEFAULT_DATAFLOW_LOCATION, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -355,18 +319,12 @@ def __init__( self.fail_on_terminal_state = fail_on_terminal_state self.location = location self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain self.hook: DataflowHook | None = None def poke(self, context: Context) -> bool: self.hook = DataflowHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) diff --git a/airflow/providers/google/cloud/sensors/dataform.py b/airflow/providers/google/cloud/sensors/dataform.py index 6a49233988ecd..965e9c5fe2cde 100644 --- a/airflow/providers/google/cloud/sensors/dataform.py +++ b/airflow/providers/google/cloud/sensors/dataform.py @@ -18,7 +18,6 @@ """This module contains a Google Cloud Dataform sensor.""" from __future__ import annotations -import warnings from typing import TYPE_CHECKING, Iterable, Sequence from airflow.exceptions import AirflowException @@ -43,10 +42,6 @@ class DataformWorkflowInvocationStateSensor(BaseSensorOperator): https://cloud.google.com/python/docs/reference/dataform/latest/google.cloud.dataform_v1beta1.types.WorkflowInvocation.State :param failure_statuses: State that will terminate the sensor with an exception :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. See: - https://developers.google.com/identity/protocols/oauth2/service-account#delegatingauthority :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -69,7 +64,6 @@ def __init__( expected_statuses: set[int] | int, failure_statuses: Iterable[int] | None = None, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -83,18 +77,12 @@ def __init__( self.project_id = project_id self.region = region self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain self.hook: DataformHook | None = None def poke(self, context: Context) -> bool: self.hook = DataformHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) diff --git a/airflow/providers/google/cloud/sensors/datafusion.py b/airflow/providers/google/cloud/sensors/datafusion.py index da25acb41aa1e..f1d441f4a8b5c 100644 --- a/airflow/providers/google/cloud/sensors/datafusion.py +++ b/airflow/providers/google/cloud/sensors/datafusion.py @@ -18,7 +18,6 @@ """This module contains a Google Cloud Data Fusion sensors.""" from __future__ import annotations -import warnings from typing import TYPE_CHECKING, Iterable, Sequence from airflow.exceptions import AirflowException, AirflowNotFoundException @@ -44,9 +43,6 @@ class CloudDataFusionPipelineStateSensor(BaseSensorOperator): is always default. If your pipeline belongs to an Enterprise edition instance, you can create a namespace. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -71,7 +67,6 @@ def __init__( project_id: str | None = None, namespace: str = "default", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -85,11 +80,6 @@ def __init__( self.project_id = project_id self.namespace = namespace self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def poke(self, context: Context) -> bool: @@ -100,7 +90,6 @@ def poke(self, context: Context) -> bool: ) hook = DataFusionHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) diff --git a/airflow/providers/google/cloud/sensors/dataplex.py b/airflow/providers/google/cloud/sensors/dataplex.py index ef828e12bb9ae..092fabd367154 100644 --- a/airflow/providers/google/cloud/sensors/dataplex.py +++ b/airflow/providers/google/cloud/sensors/dataplex.py @@ -17,7 +17,6 @@ """This module contains Google Dataplex sensors.""" from __future__ import annotations -import warnings from typing import TYPE_CHECKING, Sequence if TYPE_CHECKING: @@ -54,8 +53,6 @@ class DataplexTaskStateSensor(BaseSensorOperator): will not be retried. :param metadata: Additional metadata that is provided to the method. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate, if any. For this to work, the service accountmaking the - request must have domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -78,7 +75,6 @@ def __init__( retry: Retry | _MethodDefault = DEFAULT, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, *args, **kwargs, @@ -92,18 +88,12 @@ def __init__( self.retry = retry self.metadata = metadata self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def poke(self, context: Context) -> bool: self.log.info("Waiting for task %s to be %s", self.dataplex_task_id, TaskState.ACTIVE) hook = DataplexHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, api_version=self.api_version, impersonation_chain=self.impersonation_chain, ) diff --git a/airflow/providers/google/cloud/sensors/gcs.py b/airflow/providers/google/cloud/sensors/gcs.py index ee78ae463c41d..397a66542cb63 100644 --- a/airflow/providers/google/cloud/sensors/gcs.py +++ b/airflow/providers/google/cloud/sensors/gcs.py @@ -45,9 +45,6 @@ class GCSObjectExistenceSensor(BaseSensorOperator): storage bucket. :param google_cloud_conn_id: The connection ID to use when connecting to Google Cloud Storage. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -72,7 +69,6 @@ def __init__( bucket: str, object: str, google_cloud_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, retry: Retry = DEFAULT_RETRY, deferrable: bool = False, @@ -83,11 +79,6 @@ def __init__( self.bucket = bucket self.object = object self.google_cloud_conn_id = google_cloud_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain self.retry = retry @@ -97,7 +88,6 @@ def poke(self, context: Context) -> bool: self.log.info("Sensor checks existence of : %s, %s", self.bucket, self.object) hook = GCSHook( gcp_conn_id=self.google_cloud_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) return hook.exists(self.bucket, self.object, self.retry) @@ -115,7 +105,6 @@ def execute(self, context: Context) -> None: poke_interval=self.poke_interval, google_cloud_conn_id=self.google_cloud_conn_id, hook_params={ - "delegate_to": self.delegate_to, "impersonation_chain": self.impersonation_chain, }, ), @@ -143,9 +132,6 @@ class GCSObjectExistenceAsyncSensor(GCSObjectExistenceSensor): :param bucket: The Google Cloud Storage bucket where the object is. :param object: The name of the object to check in the Google cloud storage bucket. :param google_cloud_conn_id: The connection ID to use when connecting to Google Cloud Storage. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -190,9 +176,6 @@ class GCSObjectUpdateSensor(BaseSensorOperator): as parameter. :param google_cloud_conn_id: The connection ID to use when connecting to Google Cloud Storage. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -216,7 +199,6 @@ def __init__( object: str, ts_func: Callable = ts_function, google_cloud_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -226,18 +208,12 @@ def __init__( self.object = object self.ts_func = ts_func self.google_cloud_conn_id = google_cloud_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def poke(self, context: Context) -> bool: self.log.info("Sensor checks existence of : %s, %s", self.bucket, self.object) hook = GCSHook( gcp_conn_id=self.google_cloud_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) return hook.is_updated_after(self.bucket, self.object, self.ts_func(context)) @@ -256,9 +232,6 @@ class GCSObjectsWithPrefixExistenceSensor(BaseSensorOperator): storage bucket. :param google_cloud_conn_id: The connection ID to use when connecting to Google Cloud Storage. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -281,7 +254,6 @@ def __init__( bucket: str, prefix: str, google_cloud_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -289,11 +261,6 @@ def __init__( self.bucket = bucket self.prefix = prefix self.google_cloud_conn_id = google_cloud_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self._matches: list[str] = [] self.impersonation_chain = impersonation_chain @@ -301,7 +268,6 @@ def poke(self, context: Context) -> bool: self.log.info("Sensor checks existence of objects: %s, %s", self.bucket, self.prefix) hook = GCSHook( gcp_conn_id=self.google_cloud_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) self._matches = hook.list(self.bucket, prefix=self.prefix) @@ -346,9 +312,6 @@ class GCSUploadSessionCompleteSensor(BaseSensorOperator): when this happens. If false an error will be raised. :param google_cloud_conn_id: The connection ID to use when connecting to Google Cloud Storage. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -375,7 +338,6 @@ def __init__( previous_objects: set[str] | None = None, allow_delete: bool = True, google_cloud_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -392,11 +354,6 @@ def __init__( self.inactivity_seconds = 0 self.allow_delete = allow_delete self.google_cloud_conn_id = google_cloud_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.last_activity_time = None self.impersonation_chain = impersonation_chain self.hook: GCSHook | None = None @@ -405,7 +362,6 @@ def _get_gcs_hook(self) -> GCSHook | None: if not self.hook: self.hook = GCSHook( gcp_conn_id=self.google_cloud_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) return self.hook diff --git a/airflow/providers/google/cloud/sensors/pubsub.py b/airflow/providers/google/cloud/sensors/pubsub.py index 44600a0945302..8ab0234d5bd50 100644 --- a/airflow/providers/google/cloud/sensors/pubsub.py +++ b/airflow/providers/google/cloud/sensors/pubsub.py @@ -18,7 +18,6 @@ """This module contains a Google PubSub sensor.""" from __future__ import annotations -import warnings from typing import TYPE_CHECKING, Any, Callable, Sequence from google.cloud.pubsub_v1.types import ReceivedMessage @@ -67,9 +66,6 @@ class PubSubPullSensor(BaseSensorOperator): immediately rather than by any downstream tasks :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param messages_callback: (Optional) Callback to process received messages. It's return value will be saved to XCom. If you are pulling large messages, you probably want to provide a custom callback. @@ -101,18 +97,12 @@ def __init__( ack_messages: bool = False, gcp_conn_id: str = "google_cloud_default", messages_callback: Callable[[list[ReceivedMessage], Context], Any] | None = None, - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: super().__init__(**kwargs) self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.project_id = project_id self.subscription = subscription self.max_messages = max_messages @@ -130,7 +120,6 @@ def execute(self, context: Context) -> Any: def poke(self, context: Context) -> bool: hook = PubSubHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) diff --git a/airflow/providers/google/cloud/transfers/adls_to_gcs.py b/airflow/providers/google/cloud/transfers/adls_to_gcs.py index b4d8a5367b922..c4e9eb3035ff5 100644 --- a/airflow/providers/google/cloud/transfers/adls_to_gcs.py +++ b/airflow/providers/google/cloud/transfers/adls_to_gcs.py @@ -22,7 +22,6 @@ from __future__ import annotations import os -import warnings from tempfile import NamedTemporaryFile from typing import TYPE_CHECKING, Sequence @@ -46,9 +45,6 @@ class ADLSToGCSOperator(ADLSListOperator): :param azure_data_lake_conn_id: The connection ID to use when connecting to Azure Data Lake Storage. :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud. - :param delegate_to: Google account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param google_impersonation_chain: Optional Google service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -111,7 +107,6 @@ def __init__( dest_gcs: str, azure_data_lake_conn_id: str, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, replace: bool = False, gzip: bool = False, google_impersonation_chain: str | Sequence[str] | None = None, @@ -124,12 +119,6 @@ def __init__( self.dest_gcs = dest_gcs self.replace = replace self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'google_impersonation_chain'", - DeprecationWarning, - ) - self.delegate_to = delegate_to self.gzip = gzip self.google_impersonation_chain = google_impersonation_chain @@ -138,7 +127,6 @@ def execute(self, context: Context): files = super().execute(context) g_hook = GCSHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.google_impersonation_chain, ) diff --git a/airflow/providers/google/cloud/transfers/azure_fileshare_to_gcs.py b/airflow/providers/google/cloud/transfers/azure_fileshare_to_gcs.py index 5b8a253192704..39d14bd048140 100644 --- a/airflow/providers/google/cloud/transfers/azure_fileshare_to_gcs.py +++ b/airflow/providers/google/cloud/transfers/azure_fileshare_to_gcs.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import warnings from tempfile import NamedTemporaryFile from typing import TYPE_CHECKING, Sequence @@ -44,9 +43,6 @@ class AzureFileShareToGCSOperator(BaseOperator): :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud. :param dest_gcs: The destination Google Cloud Storage bucket and prefix where you want to store the files. (templated) - :param delegate_to: Google account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param replace: Whether you want to replace existing destination files or not. :param gzip: Option to compress file for upload @@ -79,7 +75,6 @@ def __init__( prefix: str = "", azure_fileshare_conn_id: str = "azure_fileshare_default", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, replace: bool = False, gzip: bool = False, google_impersonation_chain: str | Sequence[str] | None = None, @@ -93,12 +88,6 @@ def __init__( self.azure_fileshare_conn_id = azure_fileshare_conn_id self.gcp_conn_id = gcp_conn_id self.dest_gcs = dest_gcs - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'google_impersonation_chain'", - DeprecationWarning, - ) - self.delegate_to = delegate_to self.replace = replace self.gzip = gzip self.google_impersonation_chain = google_impersonation_chain @@ -123,7 +112,6 @@ def execute(self, context: Context): gcs_hook = GCSHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.google_impersonation_chain, ) diff --git a/airflow/providers/google/cloud/transfers/bigquery_to_bigquery.py b/airflow/providers/google/cloud/transfers/bigquery_to_bigquery.py index d70ba1e90f060..fef6f73582e9f 100644 --- a/airflow/providers/google/cloud/transfers/bigquery_to_bigquery.py +++ b/airflow/providers/google/cloud/transfers/bigquery_to_bigquery.py @@ -50,9 +50,6 @@ class BigQueryToBigQueryOperator(BaseOperator): :param write_disposition: The write disposition if the table already exists. :param create_disposition: The create disposition if the table doesn't exist. :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param labels: a dictionary containing labels for the job/query, passed to BigQuery :param encryption_configuration: [Optional] Custom encryption configuration (e.g., Cloud KMS keys). @@ -94,7 +91,6 @@ def __init__( write_disposition: str = "WRITE_EMPTY", create_disposition: str = "CREATE_IF_NEEDED", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, labels: dict | None = None, encryption_configuration: dict | None = None, location: str | None = None, @@ -108,11 +104,6 @@ def __init__( self.write_disposition = write_disposition self.create_disposition = create_disposition self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.labels = labels self.encryption_configuration = encryption_configuration self.location = location @@ -126,7 +117,6 @@ def execute(self, context: Context) -> None: ) hook = BigQueryHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, location=self.location, impersonation_chain=self.impersonation_chain, ) diff --git a/airflow/providers/google/cloud/transfers/bigquery_to_gcs.py b/airflow/providers/google/cloud/transfers/bigquery_to_gcs.py index c30cfce3c0c71..68be7e215b8bb 100644 --- a/airflow/providers/google/cloud/transfers/bigquery_to_gcs.py +++ b/airflow/providers/google/cloud/transfers/bigquery_to_gcs.py @@ -18,7 +18,6 @@ """This module contains Google BigQuery to Google Cloud Storage operator.""" from __future__ import annotations -import warnings from typing import TYPE_CHECKING, Any, Sequence from google.api_core.exceptions import Conflict @@ -60,9 +59,6 @@ class BigQueryToGCSOperator(BaseOperator): :param field_delimiter: The delimiter to use when extracting to a CSV. :param print_header: Whether to print a header for a CSV file extract. :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param labels: a dictionary containing labels for the job/query, passed to BigQuery :param location: The location used for the operation. @@ -110,7 +106,6 @@ def __init__( field_delimiter: str = ",", print_header: bool = True, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, labels: dict | None = None, location: str | None = None, impersonation_chain: str | Sequence[str] | None = None, @@ -131,11 +126,6 @@ def __init__( self.field_delimiter = field_delimiter self.print_header = print_header self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.labels = labels self.location = location self.impersonation_chain = impersonation_chain @@ -209,7 +199,6 @@ def execute(self, context: Context): ) hook = BigQueryHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, location=self.location, impersonation_chain=self.impersonation_chain, ) diff --git a/airflow/providers/google/cloud/transfers/bigquery_to_mssql.py b/airflow/providers/google/cloud/transfers/bigquery_to_mssql.py index 29bc196e8beda..095eb874c7597 100644 --- a/airflow/providers/google/cloud/transfers/bigquery_to_mssql.py +++ b/airflow/providers/google/cloud/transfers/bigquery_to_mssql.py @@ -18,7 +18,6 @@ """This module contains Google BigQuery to MSSQL operator.""" from __future__ import annotations -import warnings from typing import TYPE_CHECKING, Sequence from airflow.models import BaseOperator @@ -63,9 +62,6 @@ class BigQueryToMsSqlOperator(BaseOperator): :param selected_fields: List of fields to return (comma-separated). If unspecified, all fields are returned. :param gcp_conn_id: reference to a specific Google Cloud hook. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param mssql_conn_id: reference to a specific mssql hook :param database: name of database which overwrite defined one in connection :param replace: Whether to replace instead of insert @@ -93,7 +89,6 @@ def __init__( gcp_conn_id: str = "google_cloud_default", mssql_conn_id: str = "mssql_default", database: str | None = None, - delegate_to: str | None = None, replace: bool = False, batch_size: int = 1000, location: str | None = None, @@ -107,11 +102,6 @@ def __init__( self.database = database self.mssql_table = mssql_table self.replace = replace - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.batch_size = batch_size self.location = location self.impersonation_chain = impersonation_chain @@ -126,7 +116,6 @@ def __init__( def execute(self, context: Context) -> None: big_query_hook = BigQueryHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, location=self.location, impersonation_chain=self.impersonation_chain, ) diff --git a/airflow/providers/google/cloud/transfers/bigquery_to_mysql.py b/airflow/providers/google/cloud/transfers/bigquery_to_mysql.py index d26b36134f885..e584127c7d4a4 100644 --- a/airflow/providers/google/cloud/transfers/bigquery_to_mysql.py +++ b/airflow/providers/google/cloud/transfers/bigquery_to_mysql.py @@ -18,7 +18,6 @@ """This module contains Google BigQuery to MySQL operator.""" from __future__ import annotations -import warnings from typing import TYPE_CHECKING, Sequence from airflow.models import BaseOperator @@ -63,9 +62,6 @@ class BigQueryToMySqlOperator(BaseOperator): :param selected_fields: List of fields to return (comma-separated). If unspecified, all fields are returned. :param gcp_conn_id: reference to a specific Google Cloud hook. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param mysql_conn_id: Reference to :ref:`mysql connection id `. :param database: name of database which overwrite defined one in connection :param replace: Whether to replace instead of insert @@ -97,7 +93,6 @@ def __init__( gcp_conn_id: str = "google_cloud_default", mysql_conn_id: str = "mysql_default", database: str | None = None, - delegate_to: str | None = None, replace: bool = False, batch_size: int = 1000, location: str | None = None, @@ -111,11 +106,6 @@ def __init__( self.database = database self.mysql_table = mysql_table self.replace = replace - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.batch_size = batch_size self.location = location self.impersonation_chain = impersonation_chain @@ -127,7 +117,6 @@ def __init__( def execute(self, context: Context) -> None: big_query_hook = BigQueryHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, location=self.location, impersonation_chain=self.impersonation_chain, ) diff --git a/airflow/providers/google/cloud/transfers/calendar_to_gcs.py b/airflow/providers/google/cloud/transfers/calendar_to_gcs.py index 506185b2a9531..0fec0b36981c1 100644 --- a/airflow/providers/google/cloud/transfers/calendar_to_gcs.py +++ b/airflow/providers/google/cloud/transfers/calendar_to_gcs.py @@ -17,7 +17,6 @@ from __future__ import annotations import json -import warnings from datetime import datetime from tempfile import NamedTemporaryFile from typing import Any, Sequence @@ -65,9 +64,6 @@ class GoogleCalendarToGCSOperator(BaseOperator): :param destination_path: The Google Cloud Storage URI array for the object created by the operator. For example: ``path/to/my/files``. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -108,7 +104,6 @@ def __init__( updated_min: datetime | None = None, destination_path: str | None = None, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -133,11 +128,6 @@ def __init__( self.updated_min = updated_min self.destination_bucket = destination_bucket self.destination_path = destination_path - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def _upload_data( @@ -146,7 +136,6 @@ def _upload_data( ) -> str: gcs_hook = GCSHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) @@ -173,7 +162,6 @@ def execute(self, context): calendar_hook = GoogleCalendarHook( api_version=self.api_version, gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) diff --git a/airflow/providers/google/cloud/transfers/cassandra_to_gcs.py b/airflow/providers/google/cloud/transfers/cassandra_to_gcs.py index 10f0516766b23..fb7711dd8a8e7 100644 --- a/airflow/providers/google/cloud/transfers/cassandra_to_gcs.py +++ b/airflow/providers/google/cloud/transfers/cassandra_to_gcs.py @@ -22,7 +22,6 @@ from __future__ import annotations import json -import warnings from base64 import b64encode from datetime import datetime from decimal import Decimal @@ -67,9 +66,6 @@ class CassandraToGCSOperator(BaseOperator): :param cassandra_conn_id: Reference to a specific Cassandra hook. :param gzip: Option to compress file for upload :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -106,7 +102,6 @@ def __init__( gzip: bool = False, cassandra_conn_id: str = "cassandra_default", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, query_timeout: float | None | NotSetType = NOT_SET, encode_uuid: bool = True, @@ -121,11 +116,6 @@ def __init__( self.approx_max_file_size_bytes = approx_max_file_size_bytes self.cassandra_conn_id = cassandra_conn_id self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'mpersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.gzip = gzip self.impersonation_chain = impersonation_chain self.query_timeout = query_timeout @@ -255,7 +245,6 @@ def _upload_to_gcs(self, file_to_upload): """Upload a file (data split or schema .json file) to Google Cloud Storage.""" hook = GCSHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) hook.upload( diff --git a/airflow/providers/google/cloud/transfers/gcs_to_bigquery.py b/airflow/providers/google/cloud/transfers/gcs_to_bigquery.py index 86df198cf55c6..f009325facfbf 100644 --- a/airflow/providers/google/cloud/transfers/gcs_to_bigquery.py +++ b/airflow/providers/google/cloud/transfers/gcs_to_bigquery.py @@ -19,7 +19,6 @@ from __future__ import annotations import json -import warnings from typing import TYPE_CHECKING, Any, Sequence from google.api_core.exceptions import BadRequest, Conflict @@ -128,9 +127,6 @@ class GCSToBigQueryOperator(BaseOperator): execute() command, which in turn gets stored in XCom for future operators to use. This can be helpful with incremental loads--during future executions, you can pick up from the max ID. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param schema_update_options: Allows the schema of the destination table to be updated as a side effect of the load job. :param src_fmt_configs: configure optional fields specific to the source format @@ -208,7 +204,6 @@ def __init__( encoding="UTF-8", max_id_key=None, gcp_conn_id="google_cloud_default", - delegate_to=None, schema_update_options=(), src_fmt_configs=None, external_table=False, @@ -274,11 +269,6 @@ def __init__( self.max_id_key = max_id_key self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.schema_update_options = schema_update_options self.src_fmt_configs = src_fmt_configs @@ -324,7 +314,6 @@ def _handle_job_error(job: BigQueryJob) -> None: def execute(self, context: Context): hook = BigQueryHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, location=self.location, impersonation_chain=self.impersonation_chain, ) @@ -356,7 +345,6 @@ def execute(self, context: Context): if self.schema_object and self.source_format != "DATASTORE_BACKUP": gcs_hook = GCSHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) self.schema_fields = json.loads( @@ -467,7 +455,6 @@ def execute_complete(self, context: Context, event: dict[str, Any]): def _find_max_value_in_column(self): hook = BigQueryHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, location=self.location, impersonation_chain=self.impersonation_chain, ) diff --git a/airflow/providers/google/cloud/transfers/gcs_to_gcs.py b/airflow/providers/google/cloud/transfers/gcs_to_gcs.py index aab6d22a4deb8..c8f811c6e7d82 100644 --- a/airflow/providers/google/cloud/transfers/gcs_to_gcs.py +++ b/airflow/providers/google/cloud/transfers/gcs_to_gcs.py @@ -18,7 +18,6 @@ """This module contains a Google Cloud Storage operator.""" from __future__ import annotations -import warnings from typing import TYPE_CHECKING, Sequence from airflow.exceptions import AirflowException @@ -71,9 +70,6 @@ class GCSToGCSOperator(BaseOperator): If source_objects = ['foo/bah/'] and delimiter = '.avro', then only the 'files' in the folder 'foo/bah/' with '.avro' delimiter will be copied to the destination object. :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param last_modified_time: When specified, the objects will be copied or moved, only if they were modified after last_modified_time. If tzinfo has not been set, UTC will be assumed. @@ -188,7 +184,6 @@ def __init__( move_object=False, replace=True, gcp_conn_id="google_cloud_default", - delegate_to=None, last_modified_time=None, maximum_modified_time=None, is_older_than=None, @@ -208,11 +203,6 @@ def __init__( self.move_object = move_object self.replace = replace self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.last_modified_time = last_modified_time self.maximum_modified_time = maximum_modified_time self.is_older_than = is_older_than @@ -224,7 +214,6 @@ def execute(self, context: Context): hook = GCSHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) if self.source_objects and self.source_object: diff --git a/airflow/providers/google/cloud/transfers/gcs_to_local.py b/airflow/providers/google/cloud/transfers/gcs_to_local.py index 1c65bd5cd7f22..c02ab14575597 100644 --- a/airflow/providers/google/cloud/transfers/gcs_to_local.py +++ b/airflow/providers/google/cloud/transfers/gcs_to_local.py @@ -16,7 +16,6 @@ # under the License. from __future__ import annotations -import warnings from typing import TYPE_CHECKING, Sequence from airflow.exceptions import AirflowException @@ -52,9 +51,6 @@ class GCSToLocalFilesystemOperator(BaseOperator): the contents of the downloaded file to XCom with the key set in this parameter. If not set, the downloaded data will not be pushed to XCom. (templated) :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -85,7 +81,6 @@ def __init__( filename: str | None = None, store_to_xcom_key: str | None = None, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, file_encoding: str = "utf-8", **kwargs, @@ -109,11 +104,6 @@ def __init__( self.object_name = object_name self.store_to_xcom_key = store_to_xcom_key self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain self.file_encoding = file_encoding @@ -121,7 +111,6 @@ def execute(self, context: Context): self.log.info("Executing download: %s, %s, %s", self.bucket, self.object_name, self.filename) hook = GCSHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) diff --git a/airflow/providers/google/cloud/transfers/gcs_to_sftp.py b/airflow/providers/google/cloud/transfers/gcs_to_sftp.py index 3d8685c7c2937..0c7729e9d191a 100644 --- a/airflow/providers/google/cloud/transfers/gcs_to_sftp.py +++ b/airflow/providers/google/cloud/transfers/gcs_to_sftp.py @@ -19,7 +19,6 @@ from __future__ import annotations import os -import warnings from tempfile import NamedTemporaryFile from typing import TYPE_CHECKING, Sequence @@ -85,9 +84,6 @@ class GCSToSFTPOperator(BaseOperator): :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud. :param sftp_conn_id: The sftp connection id. The name or identifier for establishing a connection to the SFTP server. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -116,7 +112,6 @@ def __init__( move_object: bool = False, gcp_conn_id: str = "google_cloud_default", sftp_conn_id: str = "ssh_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -129,18 +124,12 @@ def __init__( self.move_object = move_object self.gcp_conn_id = gcp_conn_id self.sftp_conn_id = sftp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain self.sftp_dirs = None def execute(self, context: Context): gcs_hook = GCSHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) diff --git a/airflow/providers/google/cloud/transfers/gdrive_to_gcs.py b/airflow/providers/google/cloud/transfers/gdrive_to_gcs.py index f07b0a9a01563..1ba0051038dad 100644 --- a/airflow/providers/google/cloud/transfers/gdrive_to_gcs.py +++ b/airflow/providers/google/cloud/transfers/gdrive_to_gcs.py @@ -16,7 +16,6 @@ # under the License. from __future__ import annotations -import warnings from typing import TYPE_CHECKING, Sequence from airflow.models import BaseOperator @@ -43,9 +42,6 @@ class GoogleDriveToGCSOperator(BaseOperator): :param file_name: The name of the file residing in Google Drive :param drive_id: Optional. The id of the shared Google Drive in which the file resides. :param gcp_conn_id: The GCP connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -74,7 +70,6 @@ def __init__( folder_id: str, drive_id: str | None = None, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -85,22 +80,15 @@ def __init__( self.drive_id = drive_id self.file_name = file_name self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context): gdrive_hook = GoogleDriveHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) gcs_hook = GCSHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) file_metadata = gdrive_hook.get_file_id( diff --git a/airflow/providers/google/cloud/transfers/gdrive_to_local.py b/airflow/providers/google/cloud/transfers/gdrive_to_local.py index 661170506f4f6..7d68127f62101 100644 --- a/airflow/providers/google/cloud/transfers/gdrive_to_local.py +++ b/airflow/providers/google/cloud/transfers/gdrive_to_local.py @@ -16,7 +16,6 @@ # under the License. from __future__ import annotations -import warnings from typing import TYPE_CHECKING, Sequence from airflow.models import BaseOperator @@ -39,9 +38,6 @@ class GoogleDriveToLocalOperator(BaseOperator): :param file_name: The name of the file residing in Google Drive :param gcp_conn_id: The GCP connection ID to use when fetching connection info. :param drive_id: Optional. The id of the shared Google Drive in which the file resides. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -68,7 +64,6 @@ def __init__( folder_id: str, drive_id: str | None = None, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -78,18 +73,12 @@ def __init__( self.drive_id = drive_id self.file_name = file_name self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context): self.log.info("Executing download: %s into %s", self.file_name, self.output_file) gdrive_hook = GoogleDriveHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) file_metadata = gdrive_hook.get_file_id( diff --git a/airflow/providers/google/cloud/transfers/local_to_gcs.py b/airflow/providers/google/cloud/transfers/local_to_gcs.py index 9ce49fecd6afe..ee3a5c867ce19 100644 --- a/airflow/providers/google/cloud/transfers/local_to_gcs.py +++ b/airflow/providers/google/cloud/transfers/local_to_gcs.py @@ -19,7 +19,6 @@ from __future__ import annotations import os -import warnings from glob import glob from typing import TYPE_CHECKING, Sequence @@ -47,7 +46,6 @@ class LocalFilesystemToGCSOperator(BaseOperator): :param bucket: The bucket to upload to. (templated) :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud. :param mime_type: The mime-type string - :param delegate_to: The account to impersonate, if any :param gzip: Allows for file to be compressed and uploaded as gzip :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token @@ -74,7 +72,6 @@ def __init__( bucket, gcp_conn_id="google_cloud_default", mime_type="application/octet-stream", - delegate_to=None, gzip=False, impersonation_chain: str | Sequence[str] | None = None, **kwargs, @@ -86,11 +83,6 @@ def __init__( self.bucket = bucket self.gcp_conn_id = gcp_conn_id self.mime_type = mime_type - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.gzip = gzip self.impersonation_chain = impersonation_chain @@ -98,7 +90,6 @@ def execute(self, context: Context): """Uploads a file or list of files to Google Cloud Storage""" hook = GCSHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) diff --git a/airflow/providers/google/cloud/transfers/s3_to_gcs.py b/airflow/providers/google/cloud/transfers/s3_to_gcs.py index 8d413a89a8364..52e0357559608 100644 --- a/airflow/providers/google/cloud/transfers/s3_to_gcs.py +++ b/airflow/providers/google/cloud/transfers/s3_to_gcs.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import warnings from tempfile import NamedTemporaryFile from typing import TYPE_CHECKING, Sequence @@ -61,9 +60,6 @@ class S3ToGCSOperator(S3ListOperator): :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud. :param dest_gcs: The destination Google Cloud Storage bucket and prefix where you want to store the files. (templated) - :param delegate_to: Google account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param replace: Whether you want to replace existing destination files or not. :param gzip: Option to compress file for upload @@ -115,7 +111,6 @@ def __init__( verify=None, gcp_conn_id="google_cloud_default", dest_gcs=None, - delegate_to=None, replace=False, gzip=False, google_impersonation_chain: str | Sequence[str] | None = None, @@ -125,11 +120,6 @@ def __init__( super().__init__(bucket=bucket, prefix=prefix, delimiter=delimiter, aws_conn_id=aws_conn_id, **kwargs) self.gcp_conn_id = gcp_conn_id self.dest_gcs = dest_gcs - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.replace = replace self.verify = verify self.gzip = gzip @@ -153,7 +143,6 @@ def execute(self, context: Context): gcs_hook = GCSHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.google_impersonation_chain, ) diff --git a/airflow/providers/google/cloud/transfers/sftp_to_gcs.py b/airflow/providers/google/cloud/transfers/sftp_to_gcs.py index aaa4bee2ac4f2..394ea3e8a2003 100644 --- a/airflow/providers/google/cloud/transfers/sftp_to_gcs.py +++ b/airflow/providers/google/cloud/transfers/sftp_to_gcs.py @@ -19,7 +19,6 @@ from __future__ import annotations import os -import warnings from tempfile import NamedTemporaryFile from typing import TYPE_CHECKING, Sequence @@ -57,9 +56,6 @@ class SFTPToGCSOperator(BaseOperator): :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud. :param sftp_conn_id: The sftp connection id. The name or identifier for establishing a connection to the SFTP server. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param mime_type: The mime-type string :param gzip: Allows for file to be compressed and uploaded as gzip :param move_object: When move object is True, the object is moved instead @@ -90,7 +86,6 @@ def __init__( destination_path: str | None = None, gcp_conn_id: str = "google_cloud_default", sftp_conn_id: str = "ssh_default", - delegate_to: str | None = None, mime_type: str = "application/octet-stream", gzip: bool = False, move_object: bool = False, @@ -104,11 +99,6 @@ def __init__( self.destination_bucket = self._set_bucket_name(destination_bucket) self.gcp_conn_id = gcp_conn_id self.mime_type = mime_type - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.gzip = gzip self.sftp_conn_id = sftp_conn_id self.move_object = move_object @@ -117,7 +107,6 @@ def __init__( def execute(self, context: Context): gcs_hook = GCSHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) diff --git a/airflow/providers/google/cloud/transfers/sheets_to_gcs.py b/airflow/providers/google/cloud/transfers/sheets_to_gcs.py index 7b7f7bfb6f2d3..a54bc3a2fb276 100644 --- a/airflow/providers/google/cloud/transfers/sheets_to_gcs.py +++ b/airflow/providers/google/cloud/transfers/sheets_to_gcs.py @@ -17,7 +17,6 @@ from __future__ import annotations import csv -import warnings from tempfile import NamedTemporaryFile from typing import TYPE_CHECKING, Any, Sequence @@ -45,9 +44,6 @@ class GoogleSheetsToGCSOperator(BaseOperator): :param destination_path: The Google cloud storage URI array for the object created by the operator. For example: ``path/to/my/files``. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -74,7 +70,6 @@ def __init__( sheet_filter: list[str] | None = None, destination_path: str | None = None, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -84,11 +79,6 @@ def __init__( self.sheet_filter = sheet_filter self.destination_bucket = destination_bucket self.destination_path = destination_path - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def _upload_data( @@ -122,12 +112,10 @@ def _upload_data( def execute(self, context: Context): sheet_hook = GSheetsHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) gcs_hook = GCSHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) diff --git a/airflow/providers/google/cloud/transfers/sql_to_gcs.py b/airflow/providers/google/cloud/transfers/sql_to_gcs.py index 0513077b04b69..1531664999e4a 100644 --- a/airflow/providers/google/cloud/transfers/sql_to_gcs.py +++ b/airflow/providers/google/cloud/transfers/sql_to_gcs.py @@ -21,7 +21,6 @@ import abc import json import os -import warnings from tempfile import NamedTemporaryFile from typing import TYPE_CHECKING, Sequence @@ -65,9 +64,6 @@ class BaseSQLToGCSOperator(BaseOperator): dict. Examples could be seen: https://cloud.google.com/bigquery/docs /schemas#specifying_a_json_schema_file :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param parameters: a parameters dict that is substituted at query runtime. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token @@ -118,7 +114,6 @@ def __init__( schema: str | list | None = None, parameters: dict | None = None, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, upload_metadata: bool = False, exclude_columns: set | None = None, @@ -143,11 +138,6 @@ def __init__( self.schema = schema self.parameters = parameters self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain self.upload_metadata = upload_metadata self.exclude_columns = exclude_columns @@ -459,7 +449,6 @@ def _upload_to_gcs(self, file_to_upload): """Upload a file (data split or schema .json file) to Google Cloud Storage.""" hook = GCSHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) is_data_file = file_to_upload.get("file_name") != self.schema_filename diff --git a/airflow/providers/google/cloud/triggers/bigquery_dts.py b/airflow/providers/google/cloud/triggers/bigquery_dts.py index a16585aaf4cd4..8ab40d99df24b 100644 --- a/airflow/providers/google/cloud/triggers/bigquery_dts.py +++ b/airflow/providers/google/cloud/triggers/bigquery_dts.py @@ -18,7 +18,6 @@ from __future__ import annotations import asyncio -import warnings from typing import Any, AsyncIterator, Sequence from google.cloud.bigquery_datatransfer_v1 import TransferRun, TransferState @@ -34,9 +33,6 @@ class BigQueryDataTransferRunTrigger(BaseTrigger): :param run_id: ID of the Transfer Run which should be watched. :param poll_interval: Optional. Interval which defines how often triggers check status of the job. :param gcp_conn_id: The connection ID used to connect to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param location: BigQuery Transfer Service location for regional transfers. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token @@ -55,7 +51,6 @@ def __init__( run_id: str, poll_interval: int = 10, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, location: str | None = None, impersonation_chain: str | Sequence[str] | None = None, ): @@ -65,11 +60,6 @@ def __init__( self.run_id = run_id self.poll_interval = poll_interval self.gcp_conn_id = gcp_conn_id - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.location = location self.impersonation_chain = impersonation_chain @@ -83,7 +73,6 @@ def serialize(self) -> tuple[str, dict[str, Any]]: "run_id": self.run_id, "poll_interval": self.poll_interval, "gcp_conn_id": self.gcp_conn_id, - "delegate_to": self.delegate_to, "location": self.location, "impersonation_chain": self.impersonation_chain, }, @@ -157,7 +146,6 @@ async def run(self) -> AsyncIterator[TriggerEvent]: def _get_async_hook(self) -> AsyncBiqQueryDataTransferServiceHook: return AsyncBiqQueryDataTransferServiceHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, location=self.location, impersonation_chain=self.impersonation_chain, ) diff --git a/airflow/providers/google/cloud/triggers/cloud_build.py b/airflow/providers/google/cloud/triggers/cloud_build.py index aad757e2b392b..c40588b5cae65 100644 --- a/airflow/providers/google/cloud/triggers/cloud_build.py +++ b/airflow/providers/google/cloud/triggers/cloud_build.py @@ -17,7 +17,6 @@ from __future__ import annotations import asyncio -import warnings from typing import Any, AsyncIterator, Sequence from google.cloud.devtools.cloudbuild_v1.types import Build @@ -41,9 +40,6 @@ class CloudBuildCreateBuildTrigger(BaseTrigger): If set as a sequence, the identities from the list must grant Service Account Token Creator IAM role to the directly preceding identity, with first account from the list granting this role to the originating account (templated). - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param poll_interval: polling period in seconds to check for the status :param location: The location of the project. """ @@ -54,7 +50,6 @@ def __init__( project_id: str | None, gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, - delegate_to: str | None = None, poll_interval: float = 4.0, location: str = "global", ): @@ -63,11 +58,6 @@ def __init__( self.project_id = project_id self.gcp_conn_id = gcp_conn_id self.impersonation_chain = impersonation_chain - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.poll_interval = poll_interval self.location = location @@ -80,7 +70,6 @@ def serialize(self) -> tuple[str, dict[str, Any]]: "project_id": self.project_id, "gcp_conn_id": self.gcp_conn_id, "impersonation_chain": self.impersonation_chain, - "delegate_to": self.delegate_to, "poll_interval": self.poll_interval, "location": self.location, }, diff --git a/airflow/providers/google/cloud/triggers/cloud_composer.py b/airflow/providers/google/cloud/triggers/cloud_composer.py index 7b8540a117b4e..e6567fa02c6b5 100644 --- a/airflow/providers/google/cloud/triggers/cloud_composer.py +++ b/airflow/providers/google/cloud/triggers/cloud_composer.py @@ -19,7 +19,6 @@ from __future__ import annotations import asyncio -import warnings from typing import Any, Sequence from airflow import AirflowException @@ -37,7 +36,6 @@ def __init__( operation_name: str, gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, - delegate_to: str | None = None, pooling_period_seconds: int = 30, ): super().__init__() @@ -46,17 +44,11 @@ def __init__( self.operation_name = operation_name self.gcp_conn_id = gcp_conn_id self.impersonation_chain = impersonation_chain - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.pooling_period_seconds = pooling_period_seconds self.gcp_hook = CloudComposerAsyncHook( gcp_conn_id=self.gcp_conn_id, impersonation_chain=self.impersonation_chain, - delegate_to=self.delegate_to, ) def serialize(self) -> tuple[str, dict[str, Any]]: @@ -68,7 +60,6 @@ def serialize(self) -> tuple[str, dict[str, Any]]: "operation_name": self.operation_name, "gcp_conn_id": self.gcp_conn_id, "impersonation_chain": self.impersonation_chain, - "delegate_to": self.delegate_to, "pooling_period_seconds": self.pooling_period_seconds, }, ) diff --git a/airflow/providers/google/cloud/triggers/dataflow.py b/airflow/providers/google/cloud/triggers/dataflow.py index 5167323789953..5dfdf5106a42b 100644 --- a/airflow/providers/google/cloud/triggers/dataflow.py +++ b/airflow/providers/google/cloud/triggers/dataflow.py @@ -36,9 +36,6 @@ class TemplateJobStartTrigger(BaseTrigger): :param location: Optional. the location where job is executed. If set to None then the value of DEFAULT_DATAFLOW_LOCATION will be used :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional. Service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -57,7 +54,6 @@ def __init__( project_id: str | None, location: str = DEFAULT_DATAFLOW_LOCATION, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, poll_sleep: int = 10, impersonation_chain: str | Sequence[str] | None = None, cancel_timeout: int | None = 5 * 60, @@ -68,7 +64,6 @@ def __init__( self.job_id = job_id self.location = location self.gcp_conn_id = gcp_conn_id - self.delegate_to = delegate_to self.poll_sleep = poll_sleep self.impersonation_chain = impersonation_chain self.cancel_timeout = cancel_timeout @@ -82,7 +77,6 @@ def serialize(self) -> tuple[str, dict[str, Any]]: "job_id": self.job_id, "location": self.location, "gcp_conn_id": self.gcp_conn_id, - "delegate_to": self.delegate_to, "poll_sleep": self.poll_sleep, "impersonation_chain": self.impersonation_chain, "cancel_timeout": self.cancel_timeout, @@ -143,7 +137,6 @@ async def run(self): def _get_async_hook(self) -> AsyncDataflowHook: return AsyncDataflowHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, poll_sleep=self.poll_sleep, impersonation_chain=self.impersonation_chain, cancel_timeout=self.cancel_timeout, diff --git a/airflow/providers/google/cloud/triggers/datafusion.py b/airflow/providers/google/cloud/triggers/datafusion.py index aae69508d311d..5619d1a26f416 100644 --- a/airflow/providers/google/cloud/triggers/datafusion.py +++ b/airflow/providers/google/cloud/triggers/datafusion.py @@ -17,7 +17,6 @@ from __future__ import annotations import asyncio -import warnings from typing import Any, AsyncIterator, Sequence from airflow.providers.google.cloud.hooks.datafusion import DataFusionAsyncHook @@ -36,9 +35,6 @@ class DataFusionStartPipelineTrigger(BaseTrigger): can create a namespace. :param gcp_conn_id: Reference to google cloud connection id :param poll_interval: polling period in seconds to check for the status - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -58,7 +54,6 @@ def __init__( poll_interval: float = 3.0, gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, - delegate_to: str | None = None, success_states: list[str] | None = None, ): super().__init__() @@ -69,11 +64,6 @@ def __init__( self.poll_interval = poll_interval self.gcp_conn_id = gcp_conn_id self.impersonation_chain = impersonation_chain - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.success_states = success_states def serialize(self) -> tuple[str, dict[str, Any]]: @@ -131,6 +121,5 @@ def _get_async_hook(self) -> DataFusionAsyncHook: pipeline_name=self.pipeline_name, pipeline_id=self.pipeline_id, gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) diff --git a/airflow/providers/google/cloud/triggers/dataproc.py b/airflow/providers/google/cloud/triggers/dataproc.py index 63f0ebcae1eda..d3736afca35ce 100644 --- a/airflow/providers/google/cloud/triggers/dataproc.py +++ b/airflow/providers/google/cloud/triggers/dataproc.py @@ -38,7 +38,6 @@ def __init__( region: str, project_id: str | None = None, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, polling_interval_seconds: int = 30, ): @@ -48,13 +47,11 @@ def __init__( self.gcp_conn_id = gcp_conn_id self.impersonation_chain = impersonation_chain self.polling_interval_seconds = polling_interval_seconds - self.delegate_to = delegate_to def get_async_hook(self): return DataprocAsyncHook( gcp_conn_id=self.gcp_conn_id, impersonation_chain=self.impersonation_chain, - delegate_to=self.delegate_to, ) @@ -77,10 +74,9 @@ class DataprocSubmitTrigger(DataprocBaseTrigger): :param polling_interval_seconds: polling period in seconds to check for the status """ - def __init__(self, job_id: str, delegate_to: str | None = None, **kwargs): + def __init__(self, job_id: str, **kwargs): self.job_id = job_id - self.delegate_to = delegate_to - super().__init__(delegate_to=self.delegate_to, **kwargs) + super().__init__(**kwargs) def serialize(self): return ( @@ -90,7 +86,6 @@ def serialize(self): "project_id": self.project_id, "region": self.region, "gcp_conn_id": self.gcp_conn_id, - "delegate_to": self.delegate_to, "impersonation_chain": self.impersonation_chain, "polling_interval_seconds": self.polling_interval_seconds, }, @@ -309,7 +304,6 @@ def serialize(self): "project_id": self.project_id, "region": self.region, "gcp_conn_id": self.gcp_conn_id, - "delegate_to": self.delegate_to, "impersonation_chain": self.impersonation_chain, "polling_interval_seconds": self.polling_interval_seconds, }, diff --git a/airflow/providers/google/cloud/triggers/kubernetes_engine.py b/airflow/providers/google/cloud/triggers/kubernetes_engine.py index 0e8baec6f9f7a..cd71e0d9a32ce 100644 --- a/airflow/providers/google/cloud/triggers/kubernetes_engine.py +++ b/airflow/providers/google/cloud/triggers/kubernetes_engine.py @@ -129,7 +129,6 @@ def __init__( project_id: str | None, location: str, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, poll_interval: int = 10, ): @@ -139,7 +138,6 @@ def __init__( self.project_id = project_id self.location = location self.gcp_conn_id = gcp_conn_id - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain self.poll_interval = poll_interval @@ -154,7 +152,6 @@ def serialize(self) -> tuple[str, dict[str, Any]]: "project_id": self.project_id, "location": self.location, "gcp_conn_id": self.gcp_conn_id, - "delegate_to": self.delegate_to, "impersonation_chain": self.impersonation_chain, "poll_interval": self.poll_interval, }, @@ -209,7 +206,6 @@ def _get_hook(self) -> GKEAsyncHook: self._hook = GKEAsyncHook( gcp_conn_id=self.gcp_conn_id, location=self.location, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) return self._hook diff --git a/airflow/providers/google/cloud/triggers/mlengine.py b/airflow/providers/google/cloud/triggers/mlengine.py index 82ee78019bd34..8d42216dcd34d 100644 --- a/airflow/providers/google/cloud/triggers/mlengine.py +++ b/airflow/providers/google/cloud/triggers/mlengine.py @@ -49,7 +49,6 @@ def __init__( labels: dict[str, str] | None = None, gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, - delegate_to: str | None = None, ): super().__init__() self.log.info("Using the connection %s .", conn_id) @@ -68,7 +67,6 @@ def __init__( self.labels = labels self.gcp_conn_id = gcp_conn_id self.impersonation_chain = impersonation_chain - self.delegate_to = delegate_to def serialize(self) -> tuple[str, dict[str, Any]]: """Serializes MLEngineStartTrainingJobTrigger arguments and classpath.""" @@ -119,6 +117,5 @@ async def run(self) -> AsyncIterator["TriggerEvent"]: # type: ignore[override] def _get_async_hook(self) -> MLEngineAsyncHook: return MLEngineAsyncHook( gcp_conn_id=self.conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) diff --git a/airflow/providers/google/common/hooks/discovery_api.py b/airflow/providers/google/common/hooks/discovery_api.py index 3e26994854616..13daf7ccf7ce6 100644 --- a/airflow/providers/google/common/hooks/discovery_api.py +++ b/airflow/providers/google/common/hooks/discovery_api.py @@ -18,7 +18,6 @@ """This module allows you to connect to the Google Discovery API Service and query it.""" from __future__ import annotations -import warnings from typing import Sequence from googleapiclient.discovery import Resource, build @@ -54,16 +53,10 @@ def __init__( api_service_name: str, api_version: str, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, ) -> None: - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) self.api_service_name = api_service_name diff --git a/airflow/providers/google/firebase/hooks/firestore.py b/airflow/providers/google/firebase/hooks/firestore.py index 2cf36eabc0291..1702f6bdb94dc 100644 --- a/airflow/providers/google/firebase/hooks/firestore.py +++ b/airflow/providers/google/firebase/hooks/firestore.py @@ -19,7 +19,6 @@ from __future__ import annotations import time -import warnings from typing import Sequence from googleapiclient.discovery import build, build_from_document @@ -40,9 +39,6 @@ class CloudFirestoreHook(GoogleBaseHook): :param api_version: API version used (for example v1 or v1beta1). :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -59,16 +55,10 @@ def __init__( self, api_version: str = "v1", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, ) -> None: - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) self.api_version = api_version diff --git a/airflow/providers/google/suite/transfers/gcs_to_gdrive.py b/airflow/providers/google/suite/transfers/gcs_to_gdrive.py index 2130afb77d68b..7f8568688c7ea 100644 --- a/airflow/providers/google/suite/transfers/gcs_to_gdrive.py +++ b/airflow/providers/google/suite/transfers/gcs_to_gdrive.py @@ -95,7 +95,6 @@ def __init__( destination_object: str | None = None, move_object: bool = False, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -106,7 +105,6 @@ def __init__( self.destination_object = destination_object self.move_object = move_object self.gcp_conn_id = gcp_conn_id - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain self.gcs_hook: GCSHook | None = None self.gdrive_hook: GoogleDriveHook | None = None @@ -115,12 +113,10 @@ def execute(self, context: Context): self.gcs_hook = GCSHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) self.gdrive_hook = GoogleDriveHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) diff --git a/airflow/providers/google/suite/transfers/gcs_to_sheets.py b/airflow/providers/google/suite/transfers/gcs_to_sheets.py index 08326e91f391d..dc0e9f3933770 100644 --- a/airflow/providers/google/suite/transfers/gcs_to_sheets.py +++ b/airflow/providers/google/suite/transfers/gcs_to_sheets.py @@ -67,7 +67,6 @@ def __init__( object_name: str, spreadsheet_range: str = "Sheet1", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -78,18 +77,15 @@ def __init__( self.spreadsheet_range = spreadsheet_range self.bucket_name = bucket_name self.object_name = object_name - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Any) -> None: sheet_hook = GSheetsHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) gcs_hook = GCSHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) with NamedTemporaryFile("w+") as temp_file: diff --git a/airflow/providers/microsoft/azure/CHANGELOG.rst b/airflow/providers/microsoft/azure/CHANGELOG.rst index 47a86ca59e3de..d8a72a7523807 100644 --- a/airflow/providers/microsoft/azure/CHANGELOG.rst +++ b/airflow/providers/microsoft/azure/CHANGELOG.rst @@ -24,6 +24,16 @@ Changelog --------- +6.0.0 +...... + +Breaking changes +~~~~~~~~~~~~~~~~ + +.. warning:: + In this version of the provider, deprecated GCS hook's param ``delegate_to`` is removed from ``AzureBlobStorageToGCSOperator``. + Impersonation can be achieved instead by utilizing the ``impersonation_chain`` param. + 5.3.1 ..... diff --git a/airflow/providers/microsoft/azure/provider.yaml b/airflow/providers/microsoft/azure/provider.yaml index e3bcda6c88e1c..86c82b3fe7b30 100644 --- a/airflow/providers/microsoft/azure/provider.yaml +++ b/airflow/providers/microsoft/azure/provider.yaml @@ -22,6 +22,7 @@ description: | `Microsoft Azure `__ suspended: false versions: + - 6.0.0 - 5.3.1 - 5.3.0 - 5.2.1 diff --git a/airflow/providers/microsoft/azure/transfers/azure_blob_to_gcs.py b/airflow/providers/microsoft/azure/transfers/azure_blob_to_gcs.py index 74a933ebcf04d..77bb6c4d48b5f 100644 --- a/airflow/providers/microsoft/azure/transfers/azure_blob_to_gcs.py +++ b/airflow/providers/microsoft/azure/transfers/azure_blob_to_gcs.py @@ -18,7 +18,6 @@ from __future__ import annotations import tempfile -import warnings from typing import TYPE_CHECKING, Sequence from airflow.models import BaseOperator @@ -46,9 +45,6 @@ class AzureBlobStorageToGCSOperator(BaseOperator): :param object_name: The object name to set when uploading the file :param filename: The local file path to the file to be uploaded :param gzip: Option to compress local file or file data for upload - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -71,7 +67,6 @@ def __init__( object_name: str, filename: str, gzip: bool, - delegate_to: str | None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -85,11 +80,6 @@ def __init__( self.object_name = object_name self.filename = filename self.gzip = gzip - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain template_fields: Sequence[str] = ( @@ -105,7 +95,6 @@ def execute(self, context: Context) -> str: azure_hook = WasbHook(wasb_conn_id=self.wasb_conn_id) gcs_hook = GCSHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) diff --git a/airflow/providers/presto/CHANGELOG.rst b/airflow/providers/presto/CHANGELOG.rst index 27c6d2321730b..498d9c29fa88d 100644 --- a/airflow/providers/presto/CHANGELOG.rst +++ b/airflow/providers/presto/CHANGELOG.rst @@ -24,6 +24,17 @@ Changelog --------- +5.0.0 +...... + +Breaking changes +~~~~~~~~~~~~~~~~ + +.. warning:: + In this version of the provider, deprecated GCS hook's param ``delegate_to`` is removed from ``GCSToPrestoOperator``. + Impersonation can be achieved instead by utilizing the ``impersonation_chain`` param. + + 4.2.2 ..... diff --git a/airflow/providers/presto/provider.yaml b/airflow/providers/presto/provider.yaml index 08c68f1ed4d3a..bec2c46622272 100644 --- a/airflow/providers/presto/provider.yaml +++ b/airflow/providers/presto/provider.yaml @@ -23,6 +23,7 @@ description: | suspended: false versions: + - 5.0.0 - 4.2.2 - 4.2.1 - 4.2.0 diff --git a/airflow/providers/presto/transfers/gcs_to_presto.py b/airflow/providers/presto/transfers/gcs_to_presto.py index d4276563be068..125ab5728a7c2 100644 --- a/airflow/providers/presto/transfers/gcs_to_presto.py +++ b/airflow/providers/presto/transfers/gcs_to_presto.py @@ -20,7 +20,6 @@ import csv import json -import warnings from tempfile import NamedTemporaryFile from typing import TYPE_CHECKING, Iterable, Sequence @@ -46,9 +45,6 @@ class GCSToPrestoOperator(BaseOperator): :param presto_conn_id: destination presto connection :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud and interact with the Google Cloud Storage service. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -75,7 +71,6 @@ def __init__( gcp_conn_id: str = "google_cloud_default", schema_fields: Iterable[str] | None = None, schema_object: str | None = None, - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -87,17 +82,11 @@ def __init__( self.gcp_conn_id = gcp_conn_id self.schema_fields = schema_fields self.schema_object = schema_object - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context) -> None: gcs_hook = GCSHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) diff --git a/airflow/providers/trino/CHANGELOG.rst b/airflow/providers/trino/CHANGELOG.rst index f44bf57555664..0cbc07bdd0f10 100644 --- a/airflow/providers/trino/CHANGELOG.rst +++ b/airflow/providers/trino/CHANGELOG.rst @@ -24,6 +24,16 @@ Changelog --------- +5.0.0 +...... + +Breaking changes +~~~~~~~~~~~~~~~~ + +.. warning:: + In this version of the provider, deprecated GCS hook's param ``delegate_to`` is removed from ``GCSToPrestoOperator``. + Impersonation can be achieved instead by utilizing the ``impersonation_chain`` param. + 4.3.2 ..... diff --git a/airflow/providers/trino/provider.yaml b/airflow/providers/trino/provider.yaml index 0412897ffa173..977da07499e25 100644 --- a/airflow/providers/trino/provider.yaml +++ b/airflow/providers/trino/provider.yaml @@ -23,6 +23,7 @@ description: | suspended: false versions: + - 5.0.0 - 4.3.2 - 4.3.1 - 4.3.0 diff --git a/airflow/providers/trino/transfers/gcs_to_trino.py b/airflow/providers/trino/transfers/gcs_to_trino.py index 538d58c74dca8..46e846830e4c5 100644 --- a/airflow/providers/trino/transfers/gcs_to_trino.py +++ b/airflow/providers/trino/transfers/gcs_to_trino.py @@ -20,7 +20,6 @@ import csv import json -import warnings from tempfile import NamedTemporaryFile from typing import TYPE_CHECKING, Iterable, Sequence @@ -49,9 +48,6 @@ class GCSToTrinoOperator(BaseOperator): :param schema_fields: The names of the columns to fill in the table. If schema_fields is provided, any path provided in the schema object will be :param schema_object: JSON file with schema fields - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -78,7 +74,6 @@ def __init__( gcp_conn_id: str = "google_cloud_default", schema_fields: Iterable[str] | None = None, schema_object: str | None = None, - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -90,17 +85,11 @@ def __init__( self.gcp_conn_id = gcp_conn_id self.schema_fields = schema_fields self.schema_object = schema_object - if delegate_to: - warnings.warn( - "'delegate_to' parameter is deprecated, please use 'impersonation_chain'", DeprecationWarning - ) - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context) -> None: gcs_hook = GCSHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) diff --git a/tests/integration/providers/google/cloud/transfers/test_trino_to_gcs.py b/tests/integration/providers/google/cloud/transfers/test_trino_to_gcs.py index 24e659e85c9bd..05b16162b6945 100644 --- a/tests/integration/providers/google/cloud/transfers/test_trino_to_gcs.py +++ b/tests/integration/providers/google/cloud/transfers/test_trino_to_gcs.py @@ -104,7 +104,6 @@ def _assert_upload(bucket, obj, tmp_filename, mime_type, gzip, metadata=None): mock_trino_hook.assert_called_once_with(trino_conn_id=TRINO_CONN_ID) mock_gcs_hook.assert_called_once_with( - delegate_to=None, gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN, ) @@ -241,7 +240,6 @@ def _assert_upload(bucket, obj, tmp_filename, mime_type, gzip, metadata=None): mock_trino_hook.assert_called_once_with(trino_conn_id=TRINO_CONN_ID) mock_gcs_hook.assert_called_once_with( - delegate_to=None, gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN, ) diff --git a/tests/providers/amazon/aws/transfers/test_glacier_to_gcs.py b/tests/providers/amazon/aws/transfers/test_glacier_to_gcs.py index f3f975633942f..6f8f67a882b9e 100644 --- a/tests/providers/amazon/aws/transfers/test_glacier_to_gcs.py +++ b/tests/providers/amazon/aws/transfers/test_glacier_to_gcs.py @@ -40,7 +40,6 @@ def test_execute(self, mock_temp, hook_gcs_mock, hook_aws_mock): aws_conn_id=AWS_CONN_ID, vault_name=VAULT_NAME, gcp_conn_id=GCP_CONN_ID, - delegate_to=None, google_impersonation_chain=None, bucket_name=BUCKET_NAME, object_name=OBJECT_NAME, @@ -55,9 +54,7 @@ def test_execute(self, mock_temp, hook_gcs_mock, hook_aws_mock): vault_name=VAULT_NAME, job_id=hook_aws_mock.return_value.retrieve_inventory.return_value[JOB_ID] ) - hook_gcs_mock.assert_called_once_with( - gcp_conn_id=GCP_CONN_ID, delegate_to=None, impersonation_chain=None - ) + hook_gcs_mock.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=None) hook_gcs_mock.return_value.upload.assert_called_once_with( bucket_name=BUCKET_NAME, object_name=OBJECT_NAME, diff --git a/tests/providers/apache/beam/operators/test_beam.py b/tests/providers/apache/beam/operators/test_beam.py index 4cefb756b4de6..e6874d710c170 100644 --- a/tests/providers/apache/beam/operators/test_beam.py +++ b/tests/providers/apache/beam/operators/test_beam.py @@ -118,7 +118,6 @@ def test_exec_dataflow_runner(self, gcs_hook, dataflow_hook_mock, beam_hook_mock job_name = dataflow_hook_mock.build_dataflow_job_name.return_value dataflow_hook_mock.assert_called_once_with( gcp_conn_id=dataflow_config.gcp_conn_id, - delegate_to=dataflow_config.delegate_to, poll_sleep=dataflow_config.poll_sleep, impersonation_chain=dataflow_config.impersonation_chain, drain_pipeline=dataflow_config.drain_pipeline, @@ -239,7 +238,6 @@ def test_exec_dataflow_runner(self, gcs_hook, dataflow_hook_mock, beam_hook_mock job_name = dataflow_hook_mock.build_dataflow_job_name.return_value dataflow_hook_mock.assert_called_once_with( gcp_conn_id=dataflow_config.gcp_conn_id, - delegate_to=dataflow_config.delegate_to, poll_sleep=dataflow_config.poll_sleep, impersonation_chain=dataflow_config.impersonation_chain, drain_pipeline=dataflow_config.drain_pipeline, @@ -539,7 +537,6 @@ def test_exec_dataflow_runner_with_go_file( job_name = dataflow_hook_mock.build_dataflow_job_name.return_value dataflow_hook_mock.assert_called_once_with( gcp_conn_id=dataflow_config.gcp_conn_id, - delegate_to=dataflow_config.delegate_to, poll_sleep=dataflow_config.poll_sleep, impersonation_chain=dataflow_config.impersonation_chain, drain_pipeline=dataflow_config.drain_pipeline, @@ -649,7 +646,6 @@ def gcs_download_side_effect(bucket_name: str, object_name: str, filename: str) mock_dataflow_hook.assert_called_once_with( gcp_conn_id=dataflow_config.gcp_conn_id, - delegate_to=dataflow_config.delegate_to, poll_sleep=dataflow_config.poll_sleep, impersonation_chain=dataflow_config.impersonation_chain, drain_pipeline=dataflow_config.drain_pipeline, diff --git a/tests/providers/google/cloud/hooks/test_automl.py b/tests/providers/google/cloud/hooks/test_automl.py index 63a1ea757985f..e774ba584afff 100644 --- a/tests/providers/google/cloud/hooks/test_automl.py +++ b/tests/providers/google/cloud/hooks/test_automl.py @@ -19,6 +19,7 @@ from unittest import mock +import pytest from google.api_core.gapic_v1.method import DEFAULT from google.cloud.automl_v1beta1 import AutoMlClient @@ -50,7 +51,11 @@ MASK = {"field": "mask"} -class TestAuoMLHook: +class TestAutoMLHook: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + CloudAutoMLHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") + def setup_method(self): with mock.patch( "airflow.providers.google.cloud.hooks.automl.GoogleBaseHook.__init__", diff --git a/tests/providers/google/cloud/hooks/test_bigquery.py b/tests/providers/google/cloud/hooks/test_bigquery.py index eef00d32c4bf3..f57700b7f5627 100644 --- a/tests/providers/google/cloud/hooks/test_bigquery.py +++ b/tests/providers/google/cloud/hooks/test_bigquery.py @@ -69,6 +69,11 @@ def get_credentials_and_project_id(self): self.hook = MockedBigQueryHook() +def test_delegate_to_runtime_error(): + with pytest.raises(RuntimeError): + BigQueryHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") + + class TestBigQueryHookMethods(_BigQueryBaseTestClass): @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryConnection") @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook._authorize") diff --git a/tests/providers/google/cloud/hooks/test_bigquery_dts.py b/tests/providers/google/cloud/hooks/test_bigquery_dts.py index 7f028a195df95..d37125a5862ff 100644 --- a/tests/providers/google/cloud/hooks/test_bigquery_dts.py +++ b/tests/providers/google/cloud/hooks/test_bigquery_dts.py @@ -55,6 +55,10 @@ class TestBigQueryDataTransferHook: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + BiqQueryDataTransferServiceHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") + def setup_method(self): with async_mock.patch( "airflow.providers.google.cloud.hooks.bigquery_dts.GoogleBaseHook.__init__", @@ -128,6 +132,10 @@ def test_get_transfer_run(self, service_mock): class TestAsyncBiqQueryDataTransferServiceHook: HOOK_MODULE_PATH = "airflow.providers.google.cloud.hooks.bigquery_dts" + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + AsyncBiqQueryDataTransferServiceHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") + @pytest.fixture() def mock_client(self): with async_mock.patch( diff --git a/tests/providers/google/cloud/hooks/test_bigtable.py b/tests/providers/google/cloud/hooks/test_bigtable.py index f3f2049f797de..231198e5c7cfe 100644 --- a/tests/providers/google/cloud/hooks/test_bigtable.py +++ b/tests/providers/google/cloud/hooks/test_bigtable.py @@ -21,6 +21,7 @@ from unittest.mock import PropertyMock import google +import pytest from google.cloud.bigtable import Client, enums from google.cloud.bigtable.instance import Instance @@ -49,6 +50,10 @@ class TestBigtableHookNoDefaultProjectId: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + BigtableHook(gcp_conn_id="test", delegate_to="delegate_to") + def setup_method(self): with mock.patch( "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.__init__", diff --git a/tests/providers/google/cloud/hooks/test_cloud_build.py b/tests/providers/google/cloud/hooks/test_cloud_build.py index 01ea91d1bf7fb..2e21bf5ae9a86 100644 --- a/tests/providers/google/cloud/hooks/test_cloud_build.py +++ b/tests/providers/google/cloud/hooks/test_cloud_build.py @@ -60,6 +60,10 @@ class TestCloudBuildHook: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + CloudBuildHook(gcp_conn_id="test", delegate_to="delegate_to") + def setup_method(self): with mock.patch( "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.__init__", @@ -322,6 +326,10 @@ def test_update_build_trigger(self, get_conn): class TestAsyncHook: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + CloudBuildAsyncHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") + @pytest.fixture def hook(self): return CloudBuildAsyncHook( diff --git a/tests/providers/google/cloud/hooks/test_cloud_composer.py b/tests/providers/google/cloud/hooks/test_cloud_composer.py index 115afead2a85d..c91e9969e9f47 100644 --- a/tests/providers/google/cloud/hooks/test_cloud_composer.py +++ b/tests/providers/google/cloud/hooks/test_cloud_composer.py @@ -29,7 +29,6 @@ TEST_GCP_REGION = "global" TEST_GCP_PROJECT = "test-project" TEST_GCP_CONN_ID = "test-gcp-conn-id" -TEST_DELEGATE_TO = "test-delegate-to" TEST_ENVIRONMENT_ID = "testenvname" TEST_ENVIRONMENT = { "name": TEST_ENVIRONMENT_ID, @@ -60,6 +59,10 @@ def mock_init(*args, **kwargs): class TestCloudComposerHook: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + CloudComposerHook(gcp_conn_id="test", delegate_to="delegate_to") + def setup_method(self): with mock.patch(BASE_STRING.format("GoogleBaseHook.__init__"), new=mock_init): self.hook = CloudComposerHook(gcp_conn_id="test") @@ -196,6 +199,10 @@ def test_list_image_versions(self, mock_client) -> None: class TestCloudComposerAsyncHook: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + CloudComposerAsyncHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") + def setup_method(self, method): with async_mock.patch(BASE_STRING.format("GoogleBaseHook.__init__"), new=mock_init): self.hook = CloudComposerAsyncHook(gcp_conn_id="test") diff --git a/tests/providers/google/cloud/hooks/test_cloud_memorystore.py b/tests/providers/google/cloud/hooks/test_cloud_memorystore.py index 3a0d96fabc06a..592acb82c2362 100644 --- a/tests/providers/google/cloud/hooks/test_cloud_memorystore.py +++ b/tests/providers/google/cloud/hooks/test_cloud_memorystore.py @@ -39,7 +39,6 @@ ) TEST_GCP_CONN_ID = "test-gcp-conn-id" -TEST_DELEGATE_TO = "test-delegate-to" TEST_LOCATION = "test-location" TEST_INSTANCE_ID = "test-instance-id" TEST_PROJECT_ID = "test-project-id" @@ -58,6 +57,10 @@ class TestCloudMemorystoreWithDefaultProjectIdHook: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + CloudMemorystoreHook(gcp_conn_id="test", delegate_to="delegate_to") + def setup_method(self): with mock.patch( "airflow.providers.google.cloud.hooks.cloud_memorystore.CloudMemorystoreHook.__init__", diff --git a/tests/providers/google/cloud/hooks/test_cloud_sql.py b/tests/providers/google/cloud/hooks/test_cloud_sql.py index 103a5f6dd73f6..64bbf8dfe5223 100644 --- a/tests/providers/google/cloud/hooks/test_cloud_sql.py +++ b/tests/providers/google/cloud/hooks/test_cloud_sql.py @@ -42,6 +42,10 @@ class TestGcpSqlHookDefaultProjectId: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + CloudSQLHook(api_version="v1", gcp_conn_id="test", delegate_to="delegate_to") + def setup_method(self): with mock.patch( "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.__init__", diff --git a/tests/providers/google/cloud/hooks/test_cloud_storage_transfer_service.py b/tests/providers/google/cloud/hooks/test_cloud_storage_transfer_service.py index 3b68688e23261..d8cd3015989c0 100644 --- a/tests/providers/google/cloud/hooks/test_cloud_storage_transfer_service.py +++ b/tests/providers/google/cloud/hooks/test_cloud_storage_transfer_service.py @@ -107,6 +107,10 @@ class GCPRequestMock: class TestGCPTransferServiceHookWithPassedName: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + CloudDataTransferServiceHook(gcp_conn_id="test", delegate_to="delegate_to") + def setup_method(self): with mock.patch( "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.__init__", diff --git a/tests/providers/google/cloud/hooks/test_compute.py b/tests/providers/google/cloud/hooks/test_compute.py index ab14b027b207b..0bba949a372c7 100644 --- a/tests/providers/google/cloud/hooks/test_compute.py +++ b/tests/providers/google/cloud/hooks/test_compute.py @@ -55,6 +55,10 @@ class TestGcpComputeHookApiCall: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + ComputeEngineHook(api_version=API_VERSION, gcp_conn_id=GCP_CONN_ID, delegate_to="delegate_to") + def setup_method(self): with mock.patch( BASE_STRING.format("GoogleBaseHook.__init__"), diff --git a/tests/providers/google/cloud/hooks/test_compute_ssh.py b/tests/providers/google/cloud/hooks/test_compute_ssh.py index 60102faa62480..b1a8801eae50b 100644 --- a/tests/providers/google/cloud/hooks/test_compute_ssh.py +++ b/tests/providers/google/cloud/hooks/test_compute_ssh.py @@ -19,6 +19,8 @@ import json from unittest import mock +import pytest + from airflow.models import Connection from airflow.providers.google.cloud.hooks.compute_ssh import ComputeEngineSSHHook @@ -33,6 +35,10 @@ class TestComputeEngineHookWithPassedProjectId: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + ComputeEngineSSHHook(gcp_conn_id="gcpssh", delegate_to="delegate_to") + @mock.patch("airflow.providers.google.cloud.hooks.compute_ssh.ComputeEngineHook") @mock.patch("airflow.providers.google.cloud.hooks.compute_ssh.OSLoginHook") @mock.patch("airflow.providers.google.cloud.hooks.compute_ssh.paramiko") @@ -59,7 +65,7 @@ def test_get_conn_default_configuration( mock_paramiko.RSAKey.generate.assert_called_once_with(2048) mock_compute_hook.assert_has_calls( [ - mock.call(delegate_to=None, gcp_conn_id="google_cloud_default"), + mock.call(gcp_conn_id="google_cloud_default"), mock.call().get_instance_address( project_id=TEST_PROJECT_ID, resource_id=TEST_INSTANCE_NAME, @@ -70,7 +76,7 @@ def test_get_conn_default_configuration( ) mock_os_login_hook.assert_has_calls( [ - mock.call(delegate_to=None, gcp_conn_id="google_cloud_default"), + mock.call(gcp_conn_id="google_cloud_default"), mock.call()._get_credentials_email(), mock.call().import_ssh_public_key( ssh_public_key={"key": "NAME AYZ root", "expiration_time_usec": mock.ANY}, @@ -118,7 +124,7 @@ def test_get_conn_authorize_using_instance_metadata( mock_paramiko.RSAKey.generate.assert_called_once_with(2048) mock_compute_hook.assert_has_calls( [ - mock.call(delegate_to=None, gcp_conn_id="google_cloud_default"), + mock.call(gcp_conn_id="google_cloud_default"), mock.call().get_instance_address( project_id=TEST_PROJECT_ID, resource_id=TEST_INSTANCE_NAME, diff --git a/tests/providers/google/cloud/hooks/test_datacatalog.py b/tests/providers/google/cloud/hooks/test_datacatalog.py index 699b09a0a1c26..c82f29d28e458 100644 --- a/tests/providers/google/cloud/hooks/test_datacatalog.py +++ b/tests/providers/google/cloud/hooks/test_datacatalog.py @@ -35,7 +35,6 @@ ) TEST_GCP_CONN_ID: str = "test-gcp-conn-id" -TEST_DELEGATE_TO: str = "test-delegate-to" TEST_LOCATION: str = "europe-west-3b" TEST_ENTRY_ID: str = "test-entry-id" TEST_ENTRY: dict = {} @@ -83,6 +82,10 @@ class TestCloudDataCatalog: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + CloudDataCatalogHook(gcp_conn_id="test", delegate_to="delegate_to") + def setup_method(self): with mock.patch( "airflow.providers.google.cloud.hooks.datacatalog.CloudDataCatalogHook.__init__", diff --git a/tests/providers/google/cloud/hooks/test_dataflow.py b/tests/providers/google/cloud/hooks/test_dataflow.py index 9e91a97d4404f..29bf44581c088 100644 --- a/tests/providers/google/cloud/hooks/test_dataflow.py +++ b/tests/providers/google/cloud/hooks/test_dataflow.py @@ -183,6 +183,10 @@ def test_fn(self, *args, **kwargs): class TestDataflowHook: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + DataflowHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") + def setup_method(self): self.dataflow_hook = DataflowHook(gcp_conn_id="google_cloud_default") self.dataflow_hook.beam_hook = MagicMock() @@ -1910,6 +1914,10 @@ def func(mock_obj, return_value): class TestAsyncHook: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + AsyncDataflowHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") + @pytest.fixture def hook(self): return AsyncDataflowHook( diff --git a/tests/providers/google/cloud/hooks/test_dataform.py b/tests/providers/google/cloud/hooks/test_dataform.py index 3df82d55dfb19..02169c07147e1 100644 --- a/tests/providers/google/cloud/hooks/test_dataform.py +++ b/tests/providers/google/cloud/hooks/test_dataform.py @@ -18,6 +18,7 @@ from unittest import mock +import pytest from google.api_core.gapic_v1.method import DEFAULT from airflow.providers.google.cloud.hooks.dataform import DataformHook @@ -31,7 +32,6 @@ REPOSITORY_ID = "test_repository" WORKSPACE_ID = "test_workspace" GCP_CONN_ID = "google_cloud_default" -DELEGATE_TO = "test-delegate-to" IMPERSONATION_CHAIN = ["ACCOUNT_1", "ACCOUNT_2", "ACCOUNT_3"] COMPILATION_RESULT = { "git_commitish": "main", @@ -53,6 +53,10 @@ class TestDataflowHook: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + DataformHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") + def setup_method(self): with mock.patch( BASE_STRING.format("GoogleBaseHook.__init__"), @@ -60,7 +64,6 @@ def setup_method(self): ): self.hook = DataformHook( gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, ) diff --git a/tests/providers/google/cloud/hooks/test_datafusion.py b/tests/providers/google/cloud/hooks/test_datafusion.py index 1b2e3ccc57081..fd6e97c3de913 100644 --- a/tests/providers/google/cloud/hooks/test_datafusion.py +++ b/tests/providers/google/cloud/hooks/test_datafusion.py @@ -80,6 +80,10 @@ def session(): class TestDataFusionHook: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + DataFusionHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") + @staticmethod def mock_endpoint(get_conn_mock): return get_conn_mock.return_value.projects.return_value.locations.return_value.instances.return_value @@ -434,6 +438,10 @@ def test_get_pipeline_workflow_should_fail_if_status_not_200(self, mock_request, class TestDataFusionHookAsynch: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + DataFusionAsyncHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") + @pytest.mark.asyncio @mock.patch(HOOK_STR.format("DataFusionAsyncHook._get_link")) async def test_async_get_pipeline_should_execute_successfully(self, mocked_link, hook_async): diff --git a/tests/providers/google/cloud/hooks/test_dataplex.py b/tests/providers/google/cloud/hooks/test_dataplex.py index 8bd7509c13a50..809565d721fc3 100644 --- a/tests/providers/google/cloud/hooks/test_dataplex.py +++ b/tests/providers/google/cloud/hooks/test_dataplex.py @@ -18,6 +18,7 @@ from unittest import mock +import pytest from google.api_core.gapic_v1.method import DEFAULT from airflow.providers.google.cloud.operators.dataplex import DataplexHook @@ -33,11 +34,14 @@ DATAPLEX_TASK_ID = "testTask001" GCP_CONN_ID = "google_cloud_default" -DELEGATE_TO = "test-delegate-to" IMPERSONATION_CHAIN = ["ACCOUNT_1", "ACCOUNT_2", "ACCOUNT_3"] class TestDataplexHook: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + DataplexHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") + def setup_method(self): with mock.patch( BASE_STRING.format("GoogleBaseHook.__init__"), @@ -45,7 +49,6 @@ def setup_method(self): ): self.hook = DataplexHook( gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, ) diff --git a/tests/providers/google/cloud/hooks/test_dataproc.py b/tests/providers/google/cloud/hooks/test_dataproc.py index 2297bbcc65433..5482fa198e7e8 100644 --- a/tests/providers/google/cloud/hooks/test_dataproc.py +++ b/tests/providers/google/cloud/hooks/test_dataproc.py @@ -71,6 +71,10 @@ async def mock_awaitable(*args, **kwargs): class TestDataprocHook: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + DataprocHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") + def setup_method(self): with mock.patch(BASE_STRING.format("GoogleBaseHook.__init__"), new=mock_init): self.hook = DataprocHook(gcp_conn_id="test") @@ -496,6 +500,10 @@ def test_list_batches(self, mock_client): class TestDataprocAsyncHook: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + DataprocAsyncHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") + def setup_method(self): with mock.patch(BASE_STRING.format("GoogleBaseHook.__init__"), new=mock_init): self.hook = DataprocAsyncHook(gcp_conn_id="test") @@ -756,9 +764,7 @@ async def test_instantiate_workflow_template(self, mock_client): @async_mock.patch(DATAPROC_STRING.format("DataprocAsyncHook.get_operation")) async def test_get_operation(self, mock_client): mock_client.return_value = None - hook = DataprocAsyncHook( - gcp_conn_id="google_cloud_default", delegate_to=None, impersonation_chain=None - ) + hook = DataprocAsyncHook(gcp_conn_id="google_cloud_default", impersonation_chain=None) await hook.get_operation(region=GCP_LOCATION, operation_name="operation_name") mock_client.assert_called_once() mock_client.assert_called_with(region=GCP_LOCATION, operation_name="operation_name") diff --git a/tests/providers/google/cloud/hooks/test_dataproc_metastore.py b/tests/providers/google/cloud/hooks/test_dataproc_metastore.py index 11c06d1f63e45..b59f6a570037b 100644 --- a/tests/providers/google/cloud/hooks/test_dataproc_metastore.py +++ b/tests/providers/google/cloud/hooks/test_dataproc_metastore.py @@ -19,6 +19,7 @@ from unittest import mock +import pytest from google.api_core.gapic_v1.method import DEFAULT from airflow.providers.google.cloud.hooks.dataproc_metastore import DataprocMetastoreHook @@ -60,6 +61,10 @@ class TestDataprocMetastoreWithDefaultProjectIdHook: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + DataprocMetastoreHook(gcp_conn_id=TEST_GCP_CONN_ID, delegate_to="delegate_to") + def setup_method(self): with mock.patch( BASE_STRING.format("GoogleBaseHook.__init__"), new=mock_base_gcp_hook_default_project_id diff --git a/tests/providers/google/cloud/hooks/test_datastore.py b/tests/providers/google/cloud/hooks/test_datastore.py index 1020475353272..d17a91d36b296 100644 --- a/tests/providers/google/cloud/hooks/test_datastore.py +++ b/tests/providers/google/cloud/hooks/test_datastore.py @@ -31,13 +31,16 @@ def mock_init( self, gcp_conn_id, - delegate_to=None, impersonation_chain=None, ): pass class TestDatastoreHook: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + DatastoreHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") + def setup_method(self): with patch( "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.__init__", new=mock_init diff --git a/tests/providers/google/cloud/hooks/test_dlp.py b/tests/providers/google/cloud/hooks/test_dlp.py index 9c9faeba65039..90efdba01790c 100644 --- a/tests/providers/google/cloud/hooks/test_dlp.py +++ b/tests/providers/google/cloud/hooks/test_dlp.py @@ -54,6 +54,10 @@ class TestCloudDLPHook: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + CloudDLPHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") + def setup_method(self): with mock.patch( "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.__init__", diff --git a/tests/providers/google/cloud/hooks/test_functions.py b/tests/providers/google/cloud/hooks/test_functions.py index 7321995204783..48465fabf73b9 100644 --- a/tests/providers/google/cloud/hooks/test_functions.py +++ b/tests/providers/google/cloud/hooks/test_functions.py @@ -36,6 +36,10 @@ class TestFunctionHookNoDefaultProjectId: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + CloudFunctionsHook(api_version="v1", gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") + def setup_method(self): with mock.patch( "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.__init__", diff --git a/tests/providers/google/cloud/hooks/test_gcs.py b/tests/providers/google/cloud/hooks/test_gcs.py index ca877bf1e77c4..c70a67a66a461 100644 --- a/tests/providers/google/cloud/hooks/test_gcs.py +++ b/tests/providers/google/cloud/hooks/test_gcs.py @@ -129,6 +129,10 @@ def test_should_raise_exception_on_mutually_exclusive(self): class TestGCSHook: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + gcs.GCSHook(api_version="v1", gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") + def setup_method(self): with mock.patch( GCS_STRING.format("GoogleBaseHook.__init__"), diff --git a/tests/providers/google/cloud/hooks/test_gdm.py b/tests/providers/google/cloud/hooks/test_gdm.py index 91be09a8d0ff0..8302b6373ad2a 100644 --- a/tests/providers/google/cloud/hooks/test_gdm.py +++ b/tests/providers/google/cloud/hooks/test_gdm.py @@ -28,7 +28,6 @@ def mock_init( self, gcp_conn_id, - delegate_to=None, impersonation_chain=None, ): pass @@ -39,6 +38,10 @@ def mock_init( class TestDeploymentManagerHook: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + GoogleDeploymentManagerHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") + def setup_method(self): with mock.patch( "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.__init__", diff --git a/tests/providers/google/cloud/hooks/test_kms.py b/tests/providers/google/cloud/hooks/test_kms.py index 221e376ae900e..e789cc43abef5 100644 --- a/tests/providers/google/cloud/hooks/test_kms.py +++ b/tests/providers/google/cloud/hooks/test_kms.py @@ -21,6 +21,7 @@ from collections import namedtuple from unittest import mock +import pytest from google.api_core.gapic_v1.method import DEFAULT from airflow.providers.google.cloud.hooks.kms import CloudKMSHook @@ -50,13 +51,16 @@ def mock_init( self, gcp_conn_id, - delegate_to=None, impersonation_chain=None, ): pass class TestCloudKMSHook: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + CloudKMSHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") + def setup_method(self): with mock.patch( "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.__init__", diff --git a/tests/providers/google/cloud/hooks/test_kubernetes_engine.py b/tests/providers/google/cloud/hooks/test_kubernetes_engine.py index 6aa468bc2aa3e..a9724cbf5418d 100644 --- a/tests/providers/google/cloud/hooks/test_kubernetes_engine.py +++ b/tests/providers/google/cloud/hooks/test_kubernetes_engine.py @@ -47,12 +47,15 @@ POD_NAMESPACE = "test" ASYNC_HOOK_STRING = GKE_STRING.format("GKEAsyncHook") GCP_CONN_ID = "test-gcp-conn-id" -DELEGATE_TO = "test-delegate-to" IMPERSONATE_CHAIN = ["impersonate", "this", "test"] OPERATION_NAME = "test-operation-name" class TestGKEHookClient: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + GKEHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") + def setup_method(self): self.gke_hook = GKEHook(location=GKE_ZONE) @@ -366,7 +369,6 @@ async def test_read_logs(self, read_namespaced_pod_log, get_conn_mock, async_hoo def async_gke_hook(): return GKEAsyncHook( gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, location=GKE_ZONE, impersonation_chain=IMPERSONATE_CHAIN, ) diff --git a/tests/providers/google/cloud/hooks/test_life_sciences.py b/tests/providers/google/cloud/hooks/test_life_sciences.py index 212590b1860a8..63ebbff4ddb1e 100644 --- a/tests/providers/google/cloud/hooks/test_life_sciences.py +++ b/tests/providers/google/cloud/hooks/test_life_sciences.py @@ -48,6 +48,10 @@ class TestLifeSciencesHookWithPassedProjectId: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + LifeSciencesHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") + def setup_method(self): with mock.patch( "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.__init__", diff --git a/tests/providers/google/cloud/hooks/test_mlengine.py b/tests/providers/google/cloud/hooks/test_mlengine.py index 67332227fe8b9..f285c76acb99d 100644 --- a/tests/providers/google/cloud/hooks/test_mlengine.py +++ b/tests/providers/google/cloud/hooks/test_mlengine.py @@ -47,6 +47,10 @@ class TestMLEngineHook: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + hook.MLEngineHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") + def setup_method(self): self.hook = hook.MLEngineHook() diff --git a/tests/providers/google/cloud/hooks/test_natural_language.py b/tests/providers/google/cloud/hooks/test_natural_language.py index b9261b0059ca2..ea61a8e75b21d 100644 --- a/tests/providers/google/cloud/hooks/test_natural_language.py +++ b/tests/providers/google/cloud/hooks/test_natural_language.py @@ -20,6 +20,7 @@ from typing import Any from unittest import mock +import pytest from google.api_core.gapic_v1.method import DEFAULT from google.cloud.language_v1.proto.language_service_pb2 import Document @@ -35,6 +36,10 @@ class TestCloudNaturalLanguageHook: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + CloudNaturalLanguageHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") + def setup_method(self): with mock.patch( "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.__init__", diff --git a/tests/providers/google/cloud/hooks/test_os_login.py b/tests/providers/google/cloud/hooks/test_os_login.py index 1f49dde2d9446..b5605f4be0e5f 100644 --- a/tests/providers/google/cloud/hooks/test_os_login.py +++ b/tests/providers/google/cloud/hooks/test_os_login.py @@ -32,7 +32,6 @@ ) TEST_GCP_CONN_ID: str = "test-gcp-conn-id" -TEST_DELEGATE_TO: str = "test-delegate-to" TEST_PROJECT_ID: str = "test-project-id" TEST_PROJECT_ID_2: str = "test-project-id-2" @@ -46,6 +45,10 @@ class TestOSLoginHook: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + OSLoginHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") + def setup_method(self): with mock.patch( "airflow.providers.google.cloud.hooks.os_login.OSLoginHook.__init__", diff --git a/tests/providers/google/cloud/hooks/test_pubsub.py b/tests/providers/google/cloud/hooks/test_pubsub.py index 4ba4f98ab581f..72e08163a6e07 100644 --- a/tests/providers/google/cloud/hooks/test_pubsub.py +++ b/tests/providers/google/cloud/hooks/test_pubsub.py @@ -53,7 +53,6 @@ def mock_init( self, gcp_conn_id, - delegate_to=None, impersonation_chain=None, ): pass @@ -73,6 +72,10 @@ def _generate_messages(count) -> list[ReceivedMessage]: class TestPubSubHook: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + PubSubHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") + def setup_method(self): with mock.patch(BASE_STRING.format("GoogleBaseHook.__init__"), new=mock_init): self.pubsub_hook = PubSubHook(gcp_conn_id="test") diff --git a/tests/providers/google/cloud/hooks/test_secret_manager.py b/tests/providers/google/cloud/hooks/test_secret_manager.py index f18e112d84b92..84e4b4334ef87 100644 --- a/tests/providers/google/cloud/hooks/test_secret_manager.py +++ b/tests/providers/google/cloud/hooks/test_secret_manager.py @@ -19,6 +19,7 @@ from unittest.mock import MagicMock, patch +import pytest from google.api_core.exceptions import NotFound from google.cloud.secretmanager_v1.proto.service_pb2 import AccessSecretVersionResponse @@ -34,6 +35,10 @@ class TestSecretsManagerHook: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + SecretsManagerHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") + @patch(INTERNAL_CLIENT_PACKAGE + "._SecretManagerClient.client", return_value=MagicMock()) @patch( SECRETS_HOOK_PACKAGE + "SecretsManagerHook.get_credentials_and_project_id", diff --git a/tests/providers/google/cloud/hooks/test_spanner.py b/tests/providers/google/cloud/hooks/test_spanner.py index 9b2639569ebaa..abb346f0016f1 100644 --- a/tests/providers/google/cloud/hooks/test_spanner.py +++ b/tests/providers/google/cloud/hooks/test_spanner.py @@ -20,6 +20,8 @@ from unittest import mock from unittest.mock import PropertyMock +import pytest + from airflow.providers.google.cloud.hooks.spanner import SpannerHook from airflow.providers.google.common.consts import CLIENT_INFO from tests.providers.google.cloud.utils.base_gcp_mock import ( @@ -34,6 +36,10 @@ class TestGcpSpannerHookDefaultProjectId: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + SpannerHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") + def setup_method(self): with mock.patch( "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.__init__", diff --git a/tests/providers/google/cloud/hooks/test_speech_to_text.py b/tests/providers/google/cloud/hooks/test_speech_to_text.py index 0a8c397ee8490..3f66a6662181b 100644 --- a/tests/providers/google/cloud/hooks/test_speech_to_text.py +++ b/tests/providers/google/cloud/hooks/test_speech_to_text.py @@ -19,6 +19,7 @@ from unittest.mock import patch +import pytest from google.api_core.gapic_v1.method import DEFAULT from airflow.providers.google.cloud.hooks.speech_to_text import CloudSpeechToTextHook @@ -31,6 +32,10 @@ class TestTextToSpeechOperator: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + CloudSpeechToTextHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") + def setup_method(self): with patch( "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.__init__", diff --git a/tests/providers/google/cloud/hooks/test_stackdriver.py b/tests/providers/google/cloud/hooks/test_stackdriver.py index f9e71e8b2e49b..9fdf1859aaa04 100644 --- a/tests/providers/google/cloud/hooks/test_stackdriver.py +++ b/tests/providers/google/cloud/hooks/test_stackdriver.py @@ -20,6 +20,7 @@ import json from unittest import mock +import pytest from google.api_core.gapic_v1.method import DEFAULT from google.cloud.monitoring_v3 import AlertPolicy, NotificationChannel from google.protobuf.field_mask_pb2 import FieldMask @@ -81,6 +82,10 @@ class TestStackdriverHookMethods: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + stackdriver.StackdriverHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") + @mock.patch( "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.get_credentials_and_project_id", return_value=(CREDENTIALS, PROJECT_ID), diff --git a/tests/providers/google/cloud/hooks/test_tasks.py b/tests/providers/google/cloud/hooks/test_tasks.py index f77ead1ba08a6..9885248db45e0 100644 --- a/tests/providers/google/cloud/hooks/test_tasks.py +++ b/tests/providers/google/cloud/hooks/test_tasks.py @@ -20,6 +20,7 @@ from typing import Any from unittest import mock +import pytest from google.api_core.gapic_v1.method import DEFAULT from google.cloud.tasks_v2.types import Queue, Task @@ -47,6 +48,10 @@ class Obj: class TestCloudTasksHook: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + CloudTasksHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") + def setup_method(self): with mock.patch( "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.__init__", diff --git a/tests/providers/google/cloud/hooks/test_text_to_speech.py b/tests/providers/google/cloud/hooks/test_text_to_speech.py index 7ce6494bd545b..784a9c8c8fd98 100644 --- a/tests/providers/google/cloud/hooks/test_text_to_speech.py +++ b/tests/providers/google/cloud/hooks/test_text_to_speech.py @@ -19,6 +19,7 @@ from unittest.mock import patch +import pytest from google.api_core.gapic_v1.method import DEFAULT from airflow.providers.google.cloud.hooks.text_to_speech import CloudTextToSpeechHook @@ -31,6 +32,10 @@ class TestTextToSpeechHook: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + CloudTextToSpeechHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") + def setup_method(self): with patch( "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.__init__", diff --git a/tests/providers/google/cloud/hooks/test_translate.py b/tests/providers/google/cloud/hooks/test_translate.py index 03c9f411c74c1..068fe60fa373c 100644 --- a/tests/providers/google/cloud/hooks/test_translate.py +++ b/tests/providers/google/cloud/hooks/test_translate.py @@ -19,6 +19,8 @@ from unittest import mock +import pytest + from airflow.providers.google.cloud.hooks.translate import CloudTranslateHook from airflow.providers.google.common.consts import CLIENT_INFO from tests.providers.google.cloud.utils.base_gcp_mock import mock_base_gcp_hook_default_project_id @@ -27,6 +29,10 @@ class TestCloudTranslateHook: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + CloudTranslateHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") + def setup_method(self): with mock.patch( "airflow.providers.google.cloud.hooks.translate.CloudTranslateHook.__init__", diff --git a/tests/providers/google/cloud/hooks/test_video_intelligence.py b/tests/providers/google/cloud/hooks/test_video_intelligence.py index 4c8fac79daab2..46017de585d01 100644 --- a/tests/providers/google/cloud/hooks/test_video_intelligence.py +++ b/tests/providers/google/cloud/hooks/test_video_intelligence.py @@ -19,6 +19,7 @@ from unittest import mock +import pytest from google.api_core.gapic_v1.method import DEFAULT from google.cloud.videointelligence_v1 import enums @@ -35,6 +36,10 @@ class TestCloudVideoIntelligenceHook: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + CloudVideoIntelligenceHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") + def setup_method(self): with mock.patch( "airflow.providers.google.cloud.hooks.video_intelligence.CloudVideoIntelligenceHook.__init__", diff --git a/tests/providers/google/cloud/hooks/test_vision.py b/tests/providers/google/cloud/hooks/test_vision.py index c5b1357e19625..dbdc4b2a66519 100644 --- a/tests/providers/google/cloud/hooks/test_vision.py +++ b/tests/providers/google/cloud/hooks/test_vision.py @@ -84,6 +84,10 @@ class TestGcpVisionHook: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + CloudVisionHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") + def setup_method(self): with mock.patch( "airflow.providers.google.cloud.hooks.vision.CloudVisionHook.__init__", diff --git a/tests/providers/google/cloud/hooks/test_workflows.py b/tests/providers/google/cloud/hooks/test_workflows.py index fd2e5c57044a6..cb5267ee5129d 100644 --- a/tests/providers/google/cloud/hooks/test_workflows.py +++ b/tests/providers/google/cloud/hooks/test_workflows.py @@ -18,6 +18,8 @@ from unittest import mock +import pytest + from airflow.providers.google.cloud.hooks.workflows import WorkflowsHook from airflow.providers.google.common.consts import CLIENT_INFO @@ -50,6 +52,10 @@ def mock_init(*args, **kwargs): class TestWorkflowsHook: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + WorkflowsHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") + def setup_method(self, _): with mock.patch(BASE_PATH.format("GoogleBaseHook.__init__"), new=mock_init): self.hook = WorkflowsHook(gcp_conn_id="test") diff --git a/tests/providers/google/cloud/hooks/vertex_ai/test_auto_ml.py b/tests/providers/google/cloud/hooks/vertex_ai/test_auto_ml.py index 26b89630235ce..9c1713d058c76 100644 --- a/tests/providers/google/cloud/hooks/vertex_ai/test_auto_ml.py +++ b/tests/providers/google/cloud/hooks/vertex_ai/test_auto_ml.py @@ -19,6 +19,7 @@ from unittest import mock +import pytest from google.api_core.gapic_v1.method import DEFAULT from airflow.providers.google.cloud.hooks.vertex_ai.auto_ml import AutoMLHook @@ -109,6 +110,10 @@ def test_list_training_pipelines(self, mock_client) -> None: class TestAutoMLWithoutDefaultProjectIdHook: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + AutoMLHook(gcp_conn_id=TEST_GCP_CONN_ID, delegate_to="delegate_to") + def setup_method(self): with mock.patch( BASE_STRING.format("GoogleBaseHook.__init__"), new=mock_base_gcp_hook_no_default_project_id diff --git a/tests/providers/google/cloud/hooks/vertex_ai/test_batch_prediction_job.py b/tests/providers/google/cloud/hooks/vertex_ai/test_batch_prediction_job.py index ef328d5a987d0..a26b5e18e5d2e 100644 --- a/tests/providers/google/cloud/hooks/vertex_ai/test_batch_prediction_job.py +++ b/tests/providers/google/cloud/hooks/vertex_ai/test_batch_prediction_job.py @@ -19,6 +19,7 @@ from unittest import mock +import pytest from google.api_core.gapic_v1.method import DEFAULT from airflow.providers.google.cloud.hooks.vertex_ai.batch_prediction_job import BatchPredictionJobHook @@ -41,6 +42,10 @@ class TestBatchPredictionJobWithDefaultProjectIdHook: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + BatchPredictionJobHook(gcp_conn_id=TEST_GCP_CONN_ID, delegate_to="delegate_to") + def setup_method(self): with mock.patch( BASE_STRING.format("GoogleBaseHook.__init__"), new=mock_base_gcp_hook_default_project_id diff --git a/tests/providers/google/cloud/hooks/vertex_ai/test_custom_job.py b/tests/providers/google/cloud/hooks/vertex_ai/test_custom_job.py index 50c8d463d5ae1..839fc2b3cc5e9 100644 --- a/tests/providers/google/cloud/hooks/vertex_ai/test_custom_job.py +++ b/tests/providers/google/cloud/hooks/vertex_ai/test_custom_job.py @@ -19,6 +19,7 @@ from unittest import mock +import pytest from google.api_core.gapic_v1.method import DEFAULT from airflow.providers.google.cloud.hooks.vertex_ai.custom_job import CustomJobHook @@ -40,6 +41,10 @@ class TestCustomJobWithDefaultProjectIdHook: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + CustomJobHook(gcp_conn_id=TEST_GCP_CONN_ID, delegate_to="delegate_to") + def setup_method(self): with mock.patch( BASE_STRING.format("GoogleBaseHook.__init__"), new=mock_base_gcp_hook_default_project_id diff --git a/tests/providers/google/cloud/hooks/vertex_ai/test_dataset.py b/tests/providers/google/cloud/hooks/vertex_ai/test_dataset.py index 59a0bfce938dc..b6ad5a675ad58 100644 --- a/tests/providers/google/cloud/hooks/vertex_ai/test_dataset.py +++ b/tests/providers/google/cloud/hooks/vertex_ai/test_dataset.py @@ -19,6 +19,7 @@ from unittest import mock +import pytest from google.api_core.gapic_v1.method import DEFAULT from airflow.providers.google.cloud.hooks.vertex_ai.dataset import DatasetHook @@ -47,6 +48,10 @@ class TestVertexAIWithDefaultProjectIdHook: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + DatasetHook(gcp_conn_id=TEST_GCP_CONN_ID, delegate_to="delegate_to") + def setup_method(self): with mock.patch( BASE_STRING.format("GoogleBaseHook.__init__"), new=mock_base_gcp_hook_default_project_id diff --git a/tests/providers/google/cloud/hooks/vertex_ai/test_endpoint_service.py b/tests/providers/google/cloud/hooks/vertex_ai/test_endpoint_service.py index ae55f69481e10..56c40bbc07e09 100644 --- a/tests/providers/google/cloud/hooks/vertex_ai/test_endpoint_service.py +++ b/tests/providers/google/cloud/hooks/vertex_ai/test_endpoint_service.py @@ -19,6 +19,7 @@ from unittest import mock +import pytest from google.api_core.gapic_v1.method import DEFAULT from airflow.providers.google.cloud.hooks.vertex_ai.endpoint_service import EndpointServiceHook @@ -43,6 +44,10 @@ class TestEndpointServiceWithDefaultProjectIdHook: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + EndpointServiceHook(gcp_conn_id=TEST_GCP_CONN_ID, delegate_to="delegate_to") + def setup_method(self): with mock.patch( BASE_STRING.format("GoogleBaseHook.__init__"), new=mock_base_gcp_hook_default_project_id diff --git a/tests/providers/google/cloud/hooks/vertex_ai/test_hyperparameter_tuning_job.py b/tests/providers/google/cloud/hooks/vertex_ai/test_hyperparameter_tuning_job.py index be44128dadccb..cb62bb91e7e7e 100644 --- a/tests/providers/google/cloud/hooks/vertex_ai/test_hyperparameter_tuning_job.py +++ b/tests/providers/google/cloud/hooks/vertex_ai/test_hyperparameter_tuning_job.py @@ -19,6 +19,7 @@ from unittest import mock +import pytest from google.api_core.gapic_v1.method import DEFAULT from airflow.providers.google.cloud.hooks.vertex_ai.hyperparameter_tuning_job import ( @@ -42,6 +43,10 @@ class TestHyperparameterTuningJobWithDefaultProjectIdHook: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + HyperparameterTuningJobHook(gcp_conn_id=TEST_GCP_CONN_ID, delegate_to="delegate_to") + def setup_method(self): with mock.patch( BASE_STRING.format("GoogleBaseHook.__init__"), new=mock_base_gcp_hook_default_project_id diff --git a/tests/providers/google/cloud/hooks/vertex_ai/test_model_service.py b/tests/providers/google/cloud/hooks/vertex_ai/test_model_service.py index 883eec1e2c813..1b684fb0bd479 100644 --- a/tests/providers/google/cloud/hooks/vertex_ai/test_model_service.py +++ b/tests/providers/google/cloud/hooks/vertex_ai/test_model_service.py @@ -19,6 +19,7 @@ from unittest import mock +import pytest from google.api_core.gapic_v1.method import DEFAULT from airflow.providers.google.cloud.hooks.vertex_ai.model_service import ModelServiceHook @@ -39,6 +40,10 @@ class TestModelServiceWithDefaultProjectIdHook: + def test_delegate_to_runtime_error(self): + with pytest.raises(RuntimeError): + ModelServiceHook(gcp_conn_id=TEST_GCP_CONN_ID, delegate_to="delegate_to") + def setup_method(self): with mock.patch( BASE_STRING.format("GoogleBaseHook.__init__"), new=mock_base_gcp_hook_default_project_id diff --git a/tests/providers/google/cloud/operators/test_cloud_build.py b/tests/providers/google/cloud/operators/test_cloud_build.py index bd02f3e02e8be..b5c2622c16be2 100644 --- a/tests/providers/google/cloud/operators/test_cloud_build.py +++ b/tests/providers/google/cloud/operators/test_cloud_build.py @@ -123,7 +123,7 @@ def test_create_build(self, mock_hook): operator = CloudBuildCreateBuildOperator(build=BUILD, task_id="id") operator.execute(context=mock.MagicMock()) - mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=None, delegate_to=None) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=None) build = Build(BUILD) mock_hook.return_value.create_build_without_waiting_for_result.assert_called_once_with( build=build, project_id=None, retry=DEFAULT, timeout=None, metadata=(), location="global" @@ -434,7 +434,7 @@ def test_async_create_build_without_wait_should_execute_successfully(mock_hook): ) operator.execute(context=mock.MagicMock()) - mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=None, delegate_to=None) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=None) build = Build(BUILD) mock_hook.return_value.create_build_without_waiting_for_result.assert_called_once_with( build=build, project_id=None, retry=DEFAULT, timeout=None, metadata=(), location="global" diff --git a/tests/providers/google/cloud/operators/test_cloud_composer.py b/tests/providers/google/cloud/operators/test_cloud_composer.py index a3d0cc2a7e86f..7bda6fce5a24b 100644 --- a/tests/providers/google/cloud/operators/test_cloud_composer.py +++ b/tests/providers/google/cloud/operators/test_cloud_composer.py @@ -37,7 +37,6 @@ TEST_GCP_REGION = "global" TEST_GCP_PROJECT = "test-project" TEST_GCP_CONN_ID = "test-gcp-conn-id" -TEST_DELEGATE_TO = None TEST_IMPERSONATION_CHAIN = None TEST_ENVIRONMENT_ID = "testenvname" TEST_ENVIRONMENT = { @@ -83,7 +82,6 @@ def test_execute(self, mock_hook, to_dict_mode) -> None: mock_hook.assert_called_once_with( gcp_conn_id=TEST_GCP_CONN_ID, impersonation_chain=TEST_IMPERSONATION_CHAIN, - delegate_to=TEST_DELEGATE_TO, ) mock_hook.return_value.create_environment.assert_called_once_with( project_id=TEST_GCP_PROJECT, @@ -135,7 +133,6 @@ def test_execute(self, mock_hook) -> None: mock_hook.assert_called_once_with( gcp_conn_id=TEST_GCP_CONN_ID, impersonation_chain=TEST_IMPERSONATION_CHAIN, - delegate_to=TEST_DELEGATE_TO, ) mock_hook.return_value.delete_environment.assert_called_once_with( project_id=TEST_GCP_PROJECT, @@ -187,7 +184,6 @@ def test_execute(self, mock_hook, to_dict_mode) -> None: mock_hook.assert_called_once_with( gcp_conn_id=TEST_GCP_CONN_ID, impersonation_chain=TEST_IMPERSONATION_CHAIN, - delegate_to=TEST_DELEGATE_TO, ) mock_hook.return_value.update_environment.assert_called_once_with( project_id=TEST_GCP_PROJECT, @@ -243,7 +239,6 @@ def test_execute(self, mock_hook, to_dict_mode) -> None: mock_hook.assert_called_once_with( gcp_conn_id=TEST_GCP_CONN_ID, impersonation_chain=TEST_IMPERSONATION_CHAIN, - delegate_to=TEST_DELEGATE_TO, ) mock_hook.return_value.get_environment.assert_called_once_with( project_id=TEST_GCP_PROJECT, @@ -271,7 +266,6 @@ def test_assert_valid_hook_call(self, mock_hook) -> None: mock_hook.assert_called_once_with( gcp_conn_id=TEST_GCP_CONN_ID, impersonation_chain=TEST_IMPERSONATION_CHAIN, - delegate_to=TEST_DELEGATE_TO, ) mock_hook.return_value.list_environments.assert_called_once_with( project_id=TEST_GCP_PROJECT, @@ -300,7 +294,6 @@ def test_assert_valid_hook_call(self, mock_hook) -> None: mock_hook.assert_called_once_with( gcp_conn_id=TEST_GCP_CONN_ID, impersonation_chain=TEST_IMPERSONATION_CHAIN, - delegate_to=TEST_DELEGATE_TO, ) mock_hook.return_value.list_image_versions.assert_called_once_with( project_id=TEST_GCP_PROJECT, diff --git a/tests/providers/google/cloud/operators/test_dataflow.py b/tests/providers/google/cloud/operators/test_dataflow.py index 3440e7ee8fedc..c68938042ffb2 100644 --- a/tests/providers/google/cloud/operators/test_dataflow.py +++ b/tests/providers/google/cloud/operators/test_dataflow.py @@ -95,7 +95,6 @@ """ TEST_SQL_JOB = {"id": "test-job-id"} GCP_CONN_ID = "test_gcp_conn_id" -DELEGATE_TO = "delegating_to_something" IMPERSONATION_CHAIN = ["impersonate", "this"] CANCEL_TIMEOUT = 10 * 420 @@ -147,7 +146,6 @@ def test_exec(self, gcs_hook, dataflow_hook_mock, beam_hook_mock, mock_callback_ mock_callback_on_job_id.assert_called_once_with(on_new_job_id_callback=mock.ANY) dataflow_hook_mock.assert_called_once_with( gcp_conn_id="google_cloud_default", - delegate_to=mock.ANY, poll_sleep=POLL_SLEEP, impersonation_chain=None, drain_pipeline=False, @@ -481,7 +479,6 @@ def deferrable_operator(self): environment={"maxWorkers": 2}, deferrable=True, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, cancel_timeout=CANCEL_TIMEOUT, ) @@ -531,7 +528,6 @@ def test_validation_deferrable_params_raises_error(self): "wait_until_finished": True, "deferrable": True, "gcp_conn_id": GCP_CONN_ID, - "delegate_to": DELEGATE_TO, "impersonation_chain": IMPERSONATION_CHAIN, "cancel_timeout": CANCEL_TIMEOUT, } @@ -566,7 +562,6 @@ def test_execute(self, mock_dataflow, sync_operator): sync_operator.execute(mock.MagicMock()) mock_dataflow.assert_called_once_with( gcp_conn_id="google_cloud_default", - delegate_to=None, drain_pipeline=False, cancel_timeout=600, wait_until_finished=None, @@ -629,7 +624,6 @@ def test_execute(self, mock_hook): start_sql.execute(mock.MagicMock()) mock_hook.assert_called_once_with( gcp_conn_id="google_cloud_default", - delegate_to=None, drain_pipeline=False, impersonation_chain=None, ) diff --git a/tests/providers/google/cloud/operators/test_dataform.py b/tests/providers/google/cloud/operators/test_dataform.py index 76acfc0134884..031d99591c487 100644 --- a/tests/providers/google/cloud/operators/test_dataform.py +++ b/tests/providers/google/cloud/operators/test_dataform.py @@ -55,7 +55,6 @@ WORKSPACE = f"projects/{PROJECT_ID}/locations/{REGION}/repositories/{REPOSITORY_ID}/workspaces/{WORKSPACE_ID}" COMPILATION_RESULT_ID = "test_compilation_result_id" GCP_CONN_ID = "google_cloud_default" -DELEGATE_TO = "test-delegate-to" IMPERSONATION_CHAIN = ["ACCOUNT_1", "ACCOUNT_2", "ACCOUNT_3"] FILEPATH = "path/to/file.txt" FILE_CONTENT = b"test content" diff --git a/tests/providers/google/cloud/operators/test_dataplex.py b/tests/providers/google/cloud/operators/test_dataplex.py index 33d27ef454c3a..a5e4b3248c753 100644 --- a/tests/providers/google/cloud/operators/test_dataplex.py +++ b/tests/providers/google/cloud/operators/test_dataplex.py @@ -46,7 +46,6 @@ DATAPLEX_TASK_ID = "testTask001" GCP_CONN_ID = "google_cloud_default" -DELEGATE_TO = "test-delegate-to" API_VERSION = "v1" IMPERSONATION_CHAIN = ["ACCOUNT_1", "ACCOUNT_2", "ACCOUNT_3"] @@ -65,7 +64,6 @@ def test_execute(self, task_mock, hook_mock): validate_only=None, api_version=API_VERSION, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, ) hook_mock.return_value.wait_for_operation.return_value = None @@ -73,7 +71,6 @@ def test_execute(self, task_mock, hook_mock): op.execute(context=mock.MagicMock()) hook_mock.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, api_version=API_VERSION, impersonation_chain=IMPERSONATION_CHAIN, ) @@ -101,13 +98,11 @@ def test_execute(self, hook_mock): task_id="delete_dataplex_task", api_version=API_VERSION, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, ) op.execute(context=None) hook_mock.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, api_version=API_VERSION, impersonation_chain=IMPERSONATION_CHAIN, ) @@ -132,13 +127,11 @@ def test_execute(self, hook_mock): task_id="list_dataplex_task", api_version=API_VERSION, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, ) op.execute(context=mock.MagicMock()) hook_mock.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, api_version=API_VERSION, impersonation_chain=IMPERSONATION_CHAIN, ) @@ -168,7 +161,6 @@ def test_execute(self, task_mock, hook_mock): task_id="get_dataplex_task", api_version=API_VERSION, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, ) hook_mock.return_value.wait_for_operation.return_value = None @@ -176,7 +168,6 @@ def test_execute(self, task_mock, hook_mock): op.execute(context=mock.MagicMock()) hook_mock.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, api_version=API_VERSION, impersonation_chain=IMPERSONATION_CHAIN, ) @@ -201,13 +192,11 @@ def test_execute(self, hook_mock): task_id="delete_dataplex_lake", api_version=API_VERSION, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, ) op.execute(context=mock.MagicMock()) hook_mock.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, api_version=API_VERSION, impersonation_chain=IMPERSONATION_CHAIN, ) @@ -234,7 +223,6 @@ def test_execute(self, lake_mock, hook_mock): validate_only=None, api_version=API_VERSION, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, ) hook_mock.return_value.wait_for_operation.return_value = None @@ -242,7 +230,6 @@ def test_execute(self, lake_mock, hook_mock): op.execute(context=mock.MagicMock()) hook_mock.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, api_version=API_VERSION, impersonation_chain=IMPERSONATION_CHAIN, ) diff --git a/tests/providers/google/cloud/operators/test_datastore.py b/tests/providers/google/cloud/operators/test_datastore.py index da6a55cc9b8f2..c4197a2a69cb8 100644 --- a/tests/providers/google/cloud/operators/test_datastore.py +++ b/tests/providers/google/cloud/operators/test_datastore.py @@ -60,7 +60,7 @@ def test_execute(self, mock_hook): ) op.execute(context={"ti": mock.MagicMock()}) - mock_hook.assert_called_once_with(gcp_conn_id=CONN_ID, delegate_to=None, impersonation_chain=None) + mock_hook.assert_called_once_with(gcp_conn_id=CONN_ID, impersonation_chain=None) mock_hook.return_value.export_to_storage_bucket.assert_called_once_with( project_id=PROJECT_ID, bucket=BUCKET, @@ -89,7 +89,7 @@ def test_execute(self, mock_hook): ) op.execute(context={"ti": mock.MagicMock()}) - mock_hook.assert_called_once_with(CONN_ID, None, impersonation_chain=None) + mock_hook.assert_called_once_with(CONN_ID, impersonation_chain=None) mock_hook.return_value.import_from_storage_bucket.assert_called_once_with( project_id=PROJECT_ID, bucket=BUCKET, diff --git a/tests/providers/google/cloud/operators/test_gcs.py b/tests/providers/google/cloud/operators/test_gcs.py index e6c6eb69ae791..4e3ee66d3852a 100644 --- a/tests/providers/google/cloud/operators/test_gcs.py +++ b/tests/providers/google/cloud/operators/test_gcs.py @@ -413,13 +413,11 @@ def test_execute(self, mock_hook): delete_extra_files=True, allow_overwrite=True, gcp_conn_id="GCP_CONN_ID", - delegate_to="DELEGATE_TO", impersonation_chain=IMPERSONATION_CHAIN, ) task.execute(context=mock.MagicMock()) mock_hook.assert_called_once_with( gcp_conn_id="GCP_CONN_ID", - delegate_to="DELEGATE_TO", impersonation_chain=IMPERSONATION_CHAIN, ) mock_hook.return_value.sync.assert_called_once_with( diff --git a/tests/providers/google/cloud/operators/test_mlengine.py b/tests/providers/google/cloud/operators/test_mlengine.py index bf3636ca2b0a7..cb4faf4052a16 100644 --- a/tests/providers/google/cloud/operators/test_mlengine.py +++ b/tests/providers/google/cloud/operators/test_mlengine.py @@ -54,7 +54,6 @@ TEST_MODEL_NAME = "test-model-name" TEST_VERSION_NAME = "test-version" TEST_GCP_CONN_ID = "test-gcp-conn-id" -TEST_DELEGATE_TO = "test-delegate-to" TEST_IMPERSONATION_CHAIN = ["ACCOUNT_1", "ACCOUNT_2", "ACCOUNT_3"] TEST_MODEL = { "name": TEST_MODEL_NAME, @@ -135,8 +134,7 @@ def test_success_with_model(self, mock_hook): prediction_output = prediction_task.execute(None) mock_hook.assert_called_once_with( - "google_cloud_default", - None, + gcp_conn_id="google_cloud_default", impersonation_chain=None, ) hook_instance.create_job.assert_called_once_with( @@ -178,8 +176,7 @@ def test_success_with_version(self, mock_hook): prediction_output = prediction_task.execute(None) mock_hook.assert_called_once_with( - "google_cloud_default", - None, + gcp_conn_id="google_cloud_default", impersonation_chain=None, ) hook_instance.create_job.assert_called_once_with( @@ -216,8 +213,7 @@ def test_success_with_uri(self, mock_hook): prediction_output = prediction_task.execute(None) mock_hook.assert_called_once_with( - "google_cloud_default", - None, + gcp_conn_id="google_cloud_default", impersonation_chain=None, ) hook_instance.create_job.assert_called_once_with( @@ -290,8 +286,7 @@ def test_http_error(self, mock_hook): prediction_task.execute(None) mock_hook.assert_called_once_with( - "google_cloud_default", - None, + gcp_conn_id="google_cloud_default", impersonation_chain=None, ) hook_instance.create_job.assert_called_once_with( @@ -332,7 +327,6 @@ def test_success_cancel_training_job(self, mock_hook): mock_hook.assert_called_once_with( gcp_conn_id="google_cloud_default", - delegate_to=None, impersonation_chain=None, ) # Make sure only 'cancel_job' is invoked on hook instance @@ -355,7 +349,6 @@ def test_http_error(self, mock_hook): mock_hook.assert_called_once_with( gcp_conn_id="google_cloud_default", - delegate_to=None, impersonation_chain=None, ) # Make sure only 'cancel_job' is invoked on hook instance @@ -375,14 +368,12 @@ def test_success_create_model(self, mock_hook): model=TEST_MODEL, operation="create", gcp_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) task.execute(context=MagicMock()) mock_hook.assert_called_once_with( - delegate_to=TEST_DELEGATE_TO, gcp_conn_id=TEST_GCP_CONN_ID, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) @@ -398,14 +389,12 @@ def test_success_get_model(self, mock_hook): model=TEST_MODEL, operation="get", gcp_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) result = task.execute(context=MagicMock()) mock_hook.assert_called_once_with( - delegate_to=TEST_DELEGATE_TO, gcp_conn_id=TEST_GCP_CONN_ID, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) @@ -422,7 +411,6 @@ def test_fail(self, mock_hook): model=TEST_MODEL, operation="invalid", gcp_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, ) with pytest.raises(ValueError): task.execute(None) @@ -436,14 +424,12 @@ def test_success_create_model(self, mock_hook): project_id=TEST_PROJECT_ID, model=TEST_MODEL, gcp_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) task.execute(context=MagicMock()) mock_hook.assert_called_once_with( - delegate_to=TEST_DELEGATE_TO, gcp_conn_id=TEST_GCP_CONN_ID, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) @@ -460,14 +446,12 @@ def test_success_get_model(self, mock_hook): project_id=TEST_PROJECT_ID, model_name=TEST_MODEL_NAME, gcp_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) result = task.execute(context=MagicMock()) mock_hook.assert_called_once_with( - delegate_to=TEST_DELEGATE_TO, gcp_conn_id=TEST_GCP_CONN_ID, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) @@ -485,7 +469,6 @@ def test_success_delete_model(self, mock_hook): project_id=TEST_PROJECT_ID, model_name=TEST_MODEL_NAME, gcp_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, impersonation_chain=TEST_IMPERSONATION_CHAIN, delete_contents=True, ) @@ -493,7 +476,6 @@ def test_success_delete_model(self, mock_hook): task.execute(context=MagicMock()) mock_hook.assert_called_once_with( - delegate_to=TEST_DELEGATE_TO, gcp_conn_id=TEST_GCP_CONN_ID, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) @@ -520,7 +502,6 @@ def test_success_create_version(self, mock_hook): mock_hook.assert_called_once_with( gcp_conn_id="google_cloud_default", - delegate_to=None, impersonation_chain=None, ) # Make sure only 'create_version' is invoked on hook instance @@ -539,14 +520,12 @@ def test_success(self, mock_hook): model_name=TEST_MODEL_NAME, version=TEST_VERSION, gcp_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) task.execute(context=MagicMock()) mock_hook.assert_called_once_with( - delegate_to=TEST_DELEGATE_TO, gcp_conn_id=TEST_GCP_CONN_ID, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) @@ -562,7 +541,6 @@ def test_missing_model_name(self): model_name=None, version=TEST_VERSION, gcp_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, ) def test_missing_version(self): @@ -573,7 +551,6 @@ def test_missing_version(self): model_name=TEST_MODEL_NAME, version=None, gcp_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, ) @@ -586,14 +563,12 @@ def test_success(self, mock_hook): model_name=TEST_MODEL_NAME, version_name=TEST_VERSION_NAME, gcp_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) task.execute(context=MagicMock()) mock_hook.assert_called_once_with( - delegate_to=TEST_DELEGATE_TO, gcp_conn_id=TEST_GCP_CONN_ID, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) @@ -609,7 +584,6 @@ def test_missing_model_name(self): model_name=None, version_name=TEST_VERSION_NAME, gcp_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, ) def test_missing_version_name(self): @@ -620,7 +594,6 @@ def test_missing_version_name(self): model_name=TEST_MODEL_NAME, version_name=None, gcp_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, ) @@ -632,14 +605,12 @@ def test_success(self, mock_hook): project_id=TEST_PROJECT_ID, model_name=TEST_MODEL_NAME, gcp_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) task.execute(context=MagicMock()) mock_hook.assert_called_once_with( - delegate_to=TEST_DELEGATE_TO, gcp_conn_id=TEST_GCP_CONN_ID, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) @@ -655,7 +626,6 @@ def test_missing_model_name(self): project_id=TEST_PROJECT_ID, model_name=None, gcp_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, ) @@ -668,14 +638,12 @@ def test_success(self, mock_hook): model_name=TEST_MODEL_NAME, version_name=TEST_VERSION_NAME, gcp_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) task.execute(context=MagicMock()) mock_hook.assert_called_once_with( - delegate_to=TEST_DELEGATE_TO, gcp_conn_id=TEST_GCP_CONN_ID, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) @@ -691,7 +659,6 @@ def test_missing_version_name(self): model_name=TEST_MODEL_NAME, version_name=None, gcp_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, ) def test_missing_model_name(self): @@ -702,7 +669,6 @@ def test_missing_model_name(self): model_name=None, version_name=TEST_VERSION_NAME, gcp_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, ) @@ -741,7 +707,6 @@ def test_create_training_job_should_execute_successfully(self, mock_hook, mock_w mock_hook.assert_called_once_with( gcp_conn_id="google_cloud_default", - delegate_to=None, impersonation_chain=None, ) mock_hook.return_value.create_job_without_waiting_result.assert_called_once_with( @@ -785,7 +750,6 @@ def test_create_training_job_with_master_config_should_execute_successfully( mock_hook.assert_called_once_with( gcp_conn_id="google_cloud_default", - delegate_to=None, impersonation_chain=None, ) mock_hook.return_value.create_job_without_waiting_result.assert_called_once_with( @@ -831,7 +795,6 @@ def test_create_training_job_with_master_image_should_execute_successfully( mock_hook.assert_called_once_with( gcp_conn_id="google_cloud_default", - delegate_to=None, impersonation_chain=None, ) mock_hook.return_value.create_job_without_waiting_result.assert_called_once_with( @@ -906,7 +869,6 @@ def test_create_training_job_with_optional_args_should_execute_successfully( mock_hook.assert_called_once_with( gcp_conn_id="google_cloud_default", - delegate_to=None, impersonation_chain=None, ) mock_hook.return_value.create_job_without_waiting_result.assert_called_once_with( @@ -929,7 +891,6 @@ def test_create_training_job_when_http_error_409_should_execute_successfully( mock_hook.assert_called_once_with( gcp_conn_id="google_cloud_default", - delegate_to=None, impersonation_chain=None, ) @@ -945,7 +906,6 @@ def test_create_training_job_should_throw_exception_when_http_error_403(self, mo mock_hook.assert_called_once_with( gcp_conn_id="google_cloud_default", - delegate_to=None, impersonation_chain=None, ) @@ -963,7 +923,6 @@ def test_create_training_job_should_throw_exception_when_job_failed(self, mock_h mock_hook.assert_called_once_with( gcp_conn_id="google_cloud_default", - delegate_to=None, impersonation_chain=None, ) mock_hook.return_value.create_job_without_waiting_result.assert_called_once_with( @@ -1111,7 +1070,6 @@ def test_async_create_training_job_with_conflict_should_execute_successfully(moc mock_hook.assert_called_once_with( gcp_conn_id="google_cloud_default", - delegate_to=None, impersonation_chain=None, ) mock_hook.return_value.create_job_without_waiting_result.assert_called_once() diff --git a/tests/providers/google/cloud/operators/test_vertex_ai.py b/tests/providers/google/cloud/operators/test_vertex_ai.py index 2ebc069968600..2c5b9a694da81 100644 --- a/tests/providers/google/cloud/operators/test_vertex_ai.py +++ b/tests/providers/google/cloud/operators/test_vertex_ai.py @@ -79,7 +79,6 @@ GCP_PROJECT = "test-project" GCP_LOCATION = "test-location" GCP_CONN_ID = "test-conn" -DELEGATE_TO = "test-delegate-to" IMPERSONATION_CHAIN = ["ACCOUNT_1", "ACCOUNT_2", "ACCOUNT_3"] STAGING_BUCKET = "gs://test-vertex-ai-bucket" DISPLAY_NAME = "display_name_1" # Create random display name @@ -178,7 +177,6 @@ def test_execute(self, mock_hook): op = CreateCustomContainerTrainingJobOperator( task_id=TASK_ID, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, staging_bucket=STAGING_BUCKET, display_name=DISPLAY_NAME, @@ -198,9 +196,7 @@ def test_execute(self, mock_hook): project_id=GCP_PROJECT, ) op.execute(context={"ti": mock.MagicMock()}) - mock_hook.assert_called_once_with( - gcp_conn_id=GCP_CONN_ID, delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN - ) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) mock_hook.return_value.create_custom_container_training_job.assert_called_once_with( staging_bucket=STAGING_BUCKET, display_name=DISPLAY_NAME, @@ -263,7 +259,6 @@ def test_execute(self, mock_hook): op = CreateCustomPythonPackageTrainingJobOperator( task_id=TASK_ID, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, staging_bucket=STAGING_BUCKET, display_name=DISPLAY_NAME, @@ -284,9 +279,7 @@ def test_execute(self, mock_hook): project_id=GCP_PROJECT, ) op.execute(context={"ti": mock.MagicMock()}) - mock_hook.assert_called_once_with( - gcp_conn_id=GCP_CONN_ID, delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN - ) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) mock_hook.return_value.create_custom_python_package_training_job.assert_called_once_with( staging_bucket=STAGING_BUCKET, display_name=DISPLAY_NAME, @@ -350,7 +343,6 @@ def test_execute(self, mock_hook): op = CreateCustomTrainingJobOperator( task_id=TASK_ID, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, staging_bucket=STAGING_BUCKET, display_name=DISPLAY_NAME, @@ -364,9 +356,7 @@ def test_execute(self, mock_hook): project_id=GCP_PROJECT, ) op.execute(context={"ti": mock.MagicMock()}) - mock_hook.assert_called_once_with( - gcp_conn_id=GCP_CONN_ID, delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN - ) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) mock_hook.return_value.create_custom_training_job.assert_called_once_with( staging_bucket=STAGING_BUCKET, display_name=DISPLAY_NAME, @@ -432,13 +422,10 @@ def test_execute(self, mock_hook): timeout=TIMEOUT, metadata=METADATA, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, ) op.execute(context={}) - mock_hook.assert_called_once_with( - gcp_conn_id=GCP_CONN_ID, delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN - ) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) mock_hook.return_value.delete_training_pipeline.assert_called_once_with( training_pipeline=TRAINING_PIPELINE_ID, region=GCP_LOCATION, @@ -468,7 +455,6 @@ def test_execute(self, mock_hook): op = ListCustomTrainingJobOperator( task_id=TASK_ID, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, region=GCP_LOCATION, project_id=GCP_PROJECT, @@ -481,9 +467,7 @@ def test_execute(self, mock_hook): metadata=METADATA, ) op.execute(context={"ti": mock.MagicMock()}) - mock_hook.assert_called_once_with( - gcp_conn_id=GCP_CONN_ID, delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN - ) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) mock_hook.return_value.list_training_pipelines.assert_called_once_with( region=GCP_LOCATION, project_id=GCP_PROJECT, @@ -504,7 +488,6 @@ def test_execute(self, mock_hook, to_dict_mock): op = CreateDatasetOperator( task_id=TASK_ID, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, region=GCP_LOCATION, project_id=GCP_PROJECT, @@ -514,9 +497,7 @@ def test_execute(self, mock_hook, to_dict_mock): metadata=METADATA, ) op.execute(context={"ti": mock.MagicMock()}) - mock_hook.assert_called_once_with( - gcp_conn_id=GCP_CONN_ID, delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN - ) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) mock_hook.return_value.create_dataset.assert_called_once_with( region=GCP_LOCATION, project_id=GCP_PROJECT, @@ -534,7 +515,6 @@ def test_execute(self, mock_hook, to_dict_mock): op = DeleteDatasetOperator( task_id=TASK_ID, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, region=GCP_LOCATION, project_id=GCP_PROJECT, @@ -544,9 +524,7 @@ def test_execute(self, mock_hook, to_dict_mock): metadata=METADATA, ) op.execute(context={}) - mock_hook.assert_called_once_with( - gcp_conn_id=GCP_CONN_ID, delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN - ) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) mock_hook.return_value.delete_dataset.assert_called_once_with( region=GCP_LOCATION, project_id=GCP_PROJECT, @@ -564,7 +542,6 @@ def test_execute(self, mock_hook, to_dict_mock): op = ExportDataOperator( task_id=TASK_ID, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, region=GCP_LOCATION, project_id=GCP_PROJECT, @@ -575,9 +552,7 @@ def test_execute(self, mock_hook, to_dict_mock): metadata=METADATA, ) op.execute(context={}) - mock_hook.assert_called_once_with( - gcp_conn_id=GCP_CONN_ID, delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN - ) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) mock_hook.return_value.export_data.assert_called_once_with( region=GCP_LOCATION, project_id=GCP_PROJECT, @@ -596,7 +571,6 @@ def test_execute(self, mock_hook, to_dict_mock): op = ImportDataOperator( task_id=TASK_ID, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, region=GCP_LOCATION, project_id=GCP_PROJECT, @@ -607,9 +581,7 @@ def test_execute(self, mock_hook, to_dict_mock): metadata=METADATA, ) op.execute(context={}) - mock_hook.assert_called_once_with( - gcp_conn_id=GCP_CONN_ID, delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN - ) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) mock_hook.return_value.import_data.assert_called_once_with( region=GCP_LOCATION, project_id=GCP_PROJECT, @@ -634,7 +606,6 @@ def test_execute(self, mock_hook, to_dict_mock): op = ListDatasetsOperator( task_id=TASK_ID, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, region=GCP_LOCATION, project_id=GCP_PROJECT, @@ -648,9 +619,7 @@ def test_execute(self, mock_hook, to_dict_mock): metadata=METADATA, ) op.execute(context={"ti": mock.MagicMock()}) - mock_hook.assert_called_once_with( - gcp_conn_id=GCP_CONN_ID, delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN - ) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) mock_hook.return_value.list_datasets.assert_called_once_with( region=GCP_LOCATION, project_id=GCP_PROJECT, @@ -672,7 +641,6 @@ def test_execute(self, mock_hook, to_dict_mock): op = UpdateDatasetOperator( task_id=TASK_ID, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, project_id=GCP_PROJECT, region=GCP_LOCATION, @@ -684,9 +652,7 @@ def test_execute(self, mock_hook, to_dict_mock): metadata=METADATA, ) op.execute(context={}) - mock_hook.assert_called_once_with( - gcp_conn_id=GCP_CONN_ID, delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN - ) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) mock_hook.return_value.update_dataset.assert_called_once_with( project_id=GCP_PROJECT, region=GCP_LOCATION, @@ -707,7 +673,6 @@ def test_execute(self, mock_hook, mock_dataset): op = CreateAutoMLForecastingTrainingJobOperator( task_id=TASK_ID, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, display_name=DISPLAY_NAME, dataset_id=TEST_DATASET_ID, @@ -724,9 +689,7 @@ def test_execute(self, mock_hook, mock_dataset): project_id=GCP_PROJECT, ) op.execute(context={"ti": mock.MagicMock()}) - mock_hook.assert_called_once_with( - gcp_conn_id=GCP_CONN_ID, delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN - ) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) mock_dataset.assert_called_once_with(dataset_name=TEST_DATASET_ID) mock_hook.return_value.create_auto_ml_forecasting_training_job.assert_called_once_with( project_id=GCP_PROJECT, @@ -774,7 +737,6 @@ def test_execute(self, mock_hook, mock_dataset): op = CreateAutoMLImageTrainingJobOperator( task_id=TASK_ID, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, display_name=DISPLAY_NAME, dataset_id=TEST_DATASET_ID, @@ -786,9 +748,7 @@ def test_execute(self, mock_hook, mock_dataset): project_id=GCP_PROJECT, ) op.execute(context={"ti": mock.MagicMock()}) - mock_hook.assert_called_once_with( - gcp_conn_id=GCP_CONN_ID, delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN - ) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) mock_dataset.assert_called_once_with(dataset_name=TEST_DATASET_ID) mock_hook.return_value.create_auto_ml_image_training_job.assert_called_once_with( project_id=GCP_PROJECT, @@ -824,7 +784,6 @@ def test_execute(self, mock_hook, mock_dataset): op = CreateAutoMLTabularTrainingJobOperator( task_id=TASK_ID, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, display_name=DISPLAY_NAME, dataset_id=TEST_DATASET_ID, @@ -835,9 +794,7 @@ def test_execute(self, mock_hook, mock_dataset): project_id=GCP_PROJECT, ) op.execute(context={"ti": mock.MagicMock()}) - mock_hook.assert_called_once_with( - gcp_conn_id=GCP_CONN_ID, delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN - ) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) mock_dataset.assert_called_once_with(dataset_name=TEST_DATASET_ID) mock_hook.return_value.create_auto_ml_tabular_training_job.assert_called_once_with( project_id=GCP_PROJECT, @@ -879,7 +836,6 @@ def test_execute(self, mock_hook, mock_dataset): op = CreateAutoMLTextTrainingJobOperator( task_id=TASK_ID, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, display_name=DISPLAY_NAME, dataset_id=TEST_DATASET_ID, @@ -891,9 +847,7 @@ def test_execute(self, mock_hook, mock_dataset): project_id=GCP_PROJECT, ) op.execute(context={"ti": mock.MagicMock()}) - mock_hook.assert_called_once_with( - gcp_conn_id=GCP_CONN_ID, delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN - ) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) mock_dataset.assert_called_once_with(dataset_name=TEST_DATASET_ID) mock_hook.return_value.create_auto_ml_text_training_job.assert_called_once_with( project_id=GCP_PROJECT, @@ -926,7 +880,6 @@ def test_execute(self, mock_hook, mock_dataset): op = CreateAutoMLVideoTrainingJobOperator( task_id=TASK_ID, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, display_name=DISPLAY_NAME, dataset_id=TEST_DATASET_ID, @@ -937,9 +890,7 @@ def test_execute(self, mock_hook, mock_dataset): project_id=GCP_PROJECT, ) op.execute(context={"ti": mock.MagicMock()}) - mock_hook.assert_called_once_with( - gcp_conn_id=GCP_CONN_ID, delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN - ) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) mock_dataset.assert_called_once_with(dataset_name=TEST_DATASET_ID) mock_hook.return_value.create_auto_ml_video_training_job.assert_called_once_with( project_id=GCP_PROJECT, @@ -973,13 +924,10 @@ def test_execute(self, mock_hook): timeout=TIMEOUT, metadata=METADATA, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, ) op.execute(context={}) - mock_hook.assert_called_once_with( - gcp_conn_id=GCP_CONN_ID, delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN - ) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) mock_hook.return_value.delete_training_pipeline.assert_called_once_with( training_pipeline=TRAINING_PIPELINE_ID, region=GCP_LOCATION, @@ -1001,7 +949,6 @@ def test_execute(self, mock_hook): op = ListAutoMLTrainingJobOperator( task_id=TASK_ID, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, region=GCP_LOCATION, project_id=GCP_PROJECT, @@ -1014,9 +961,7 @@ def test_execute(self, mock_hook): metadata=METADATA, ) op.execute(context={"ti": mock.MagicMock()}) - mock_hook.assert_called_once_with( - gcp_conn_id=GCP_CONN_ID, delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN - ) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) mock_hook.return_value.list_training_pipelines.assert_called_once_with( region=GCP_LOCATION, project_id=GCP_PROJECT, @@ -1037,7 +982,6 @@ def test_execute(self, mock_hook, to_dict_mock): op = CreateBatchPredictionJobOperator( task_id=TASK_ID, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, region=GCP_LOCATION, project_id=GCP_PROJECT, @@ -1047,9 +991,7 @@ def test_execute(self, mock_hook, to_dict_mock): predictions_format="jsonl", ) op.execute(context={"ti": mock.MagicMock()}) - mock_hook.assert_called_once_with( - gcp_conn_id=GCP_CONN_ID, delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN - ) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) mock_hook.return_value.create_batch_prediction_job.assert_called_once_with( region=GCP_LOCATION, project_id=GCP_PROJECT, @@ -1082,7 +1024,6 @@ def test_execute(self, mock_hook): op = DeleteBatchPredictionJobOperator( task_id=TASK_ID, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, region=GCP_LOCATION, project_id=GCP_PROJECT, @@ -1092,9 +1033,7 @@ def test_execute(self, mock_hook): metadata=METADATA, ) op.execute(context={}) - mock_hook.assert_called_once_with( - gcp_conn_id=GCP_CONN_ID, delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN - ) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) mock_hook.return_value.delete_batch_prediction_job.assert_called_once_with( region=GCP_LOCATION, project_id=GCP_PROJECT, @@ -1116,7 +1055,6 @@ def test_execute(self, mock_hook): op = ListBatchPredictionJobsOperator( task_id=TASK_ID, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, region=GCP_LOCATION, project_id=GCP_PROJECT, @@ -1129,9 +1067,7 @@ def test_execute(self, mock_hook): metadata=METADATA, ) op.execute(context={"ti": mock.MagicMock()}) - mock_hook.assert_called_once_with( - gcp_conn_id=GCP_CONN_ID, delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN - ) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) mock_hook.return_value.list_batch_prediction_jobs.assert_called_once_with( region=GCP_LOCATION, project_id=GCP_PROJECT, @@ -1152,7 +1088,6 @@ def test_execute(self, mock_hook, to_dict_mock): op = CreateEndpointOperator( task_id=TASK_ID, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, region=GCP_LOCATION, project_id=GCP_PROJECT, @@ -1163,9 +1098,7 @@ def test_execute(self, mock_hook, to_dict_mock): metadata=METADATA, ) op.execute(context={"ti": mock.MagicMock()}) - mock_hook.assert_called_once_with( - gcp_conn_id=GCP_CONN_ID, delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN - ) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) mock_hook.return_value.create_endpoint.assert_called_once_with( region=GCP_LOCATION, project_id=GCP_PROJECT, @@ -1183,7 +1116,6 @@ def test_execute(self, mock_hook): op = DeleteEndpointOperator( task_id=TASK_ID, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, region=GCP_LOCATION, project_id=GCP_PROJECT, @@ -1193,9 +1125,7 @@ def test_execute(self, mock_hook): metadata=METADATA, ) op.execute(context={}) - mock_hook.assert_called_once_with( - gcp_conn_id=GCP_CONN_ID, delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN - ) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) mock_hook.return_value.delete_endpoint.assert_called_once_with( region=GCP_LOCATION, project_id=GCP_PROJECT, @@ -1213,7 +1143,6 @@ def test_execute(self, mock_hook, to_dict_mock): op = DeployModelOperator( task_id=TASK_ID, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, region=GCP_LOCATION, project_id=GCP_PROJECT, @@ -1225,9 +1154,7 @@ def test_execute(self, mock_hook, to_dict_mock): metadata=METADATA, ) op.execute(context={"ti": mock.MagicMock()}) - mock_hook.assert_called_once_with( - gcp_conn_id=GCP_CONN_ID, delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN - ) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) mock_hook.return_value.deploy_model.assert_called_once_with( region=GCP_LOCATION, project_id=GCP_PROJECT, @@ -1252,7 +1179,6 @@ def test_execute(self, mock_hook): op = ListEndpointsOperator( task_id=TASK_ID, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, region=GCP_LOCATION, project_id=GCP_PROJECT, @@ -1266,9 +1192,7 @@ def test_execute(self, mock_hook): metadata=METADATA, ) op.execute(context={"ti": mock.MagicMock()}) - mock_hook.assert_called_once_with( - gcp_conn_id=GCP_CONN_ID, delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN - ) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) mock_hook.return_value.list_endpoints.assert_called_once_with( region=GCP_LOCATION, project_id=GCP_PROJECT, @@ -1289,7 +1213,6 @@ def test_execute(self, mock_hook): op = UndeployModelOperator( task_id=TASK_ID, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, region=GCP_LOCATION, project_id=GCP_PROJECT, @@ -1301,9 +1224,7 @@ def test_execute(self, mock_hook): metadata=METADATA, ) op.execute(context={}) - mock_hook.assert_called_once_with( - gcp_conn_id=GCP_CONN_ID, delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN - ) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) mock_hook.return_value.undeploy_model.assert_called_once_with( region=GCP_LOCATION, project_id=GCP_PROJECT, @@ -1323,7 +1244,6 @@ def test_execute(self, mock_hook, to_dict_mock): op = CreateHyperparameterTuningJobOperator( task_id=TASK_ID, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, region=GCP_LOCATION, project_id=GCP_PROJECT, @@ -1337,9 +1257,7 @@ def test_execute(self, mock_hook, to_dict_mock): parallel_trial_count=3, ) op.execute(context={"ti": mock.MagicMock()}) - mock_hook.assert_called_once_with( - gcp_conn_id=GCP_CONN_ID, delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN - ) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) mock_hook.return_value.create_hyperparameter_tuning_job.assert_called_once_with( project_id=GCP_PROJECT, region=GCP_LOCATION, @@ -1375,16 +1293,13 @@ def test_execute(self, mock_hook, to_dict_mock): op = GetHyperparameterTuningJobOperator( task_id=TASK_ID, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, region=GCP_LOCATION, project_id=GCP_PROJECT, hyperparameter_tuning_job_id=TEST_HYPERPARAMETER_TUNING_JOB_ID, ) op.execute(context={"ti": mock.MagicMock()}) - mock_hook.assert_called_once_with( - gcp_conn_id=GCP_CONN_ID, delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN - ) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) mock_hook.return_value.get_hyperparameter_tuning_job.assert_called_once_with( project_id=GCP_PROJECT, region=GCP_LOCATION, @@ -1401,16 +1316,13 @@ def test_execute(self, mock_hook): op = DeleteHyperparameterTuningJobOperator( task_id=TASK_ID, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, region=GCP_LOCATION, project_id=GCP_PROJECT, hyperparameter_tuning_job_id=TEST_HYPERPARAMETER_TUNING_JOB_ID, ) op.execute(context={}) - mock_hook.assert_called_once_with( - gcp_conn_id=GCP_CONN_ID, delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN - ) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) mock_hook.return_value.delete_hyperparameter_tuning_job.assert_called_once_with( project_id=GCP_PROJECT, region=GCP_LOCATION, @@ -1432,7 +1344,6 @@ def test_execute(self, mock_hook): op = ListHyperparameterTuningJobOperator( task_id=TASK_ID, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, region=GCP_LOCATION, project_id=GCP_PROJECT, @@ -1445,9 +1356,7 @@ def test_execute(self, mock_hook): metadata=METADATA, ) op.execute(context={"ti": mock.MagicMock()}) - mock_hook.assert_called_once_with( - gcp_conn_id=GCP_CONN_ID, delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN - ) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) mock_hook.return_value.list_hyperparameter_tuning_jobs.assert_called_once_with( region=GCP_LOCATION, project_id=GCP_PROJECT, @@ -1467,7 +1376,6 @@ def test_execute(self, mock_hook): op = ExportModelOperator( task_id=TASK_ID, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, region=GCP_LOCATION, project_id=GCP_PROJECT, @@ -1478,9 +1386,7 @@ def test_execute(self, mock_hook): metadata=METADATA, ) op.execute(context={"ti": mock.MagicMock()}) - mock_hook.assert_called_once_with( - gcp_conn_id=GCP_CONN_ID, delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN - ) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) mock_hook.return_value.export_model.assert_called_once_with( region=GCP_LOCATION, project_id=GCP_PROJECT, @@ -1498,7 +1404,6 @@ def test_execute(self, mock_hook): op = DeleteModelOperator( task_id=TASK_ID, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, region=GCP_LOCATION, project_id=GCP_PROJECT, @@ -1508,9 +1413,7 @@ def test_execute(self, mock_hook): metadata=METADATA, ) op.execute(context={}) - mock_hook.assert_called_once_with( - gcp_conn_id=GCP_CONN_ID, delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN - ) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) mock_hook.return_value.delete_model.assert_called_once_with( region=GCP_LOCATION, project_id=GCP_PROJECT, @@ -1534,7 +1437,6 @@ def test_execute(self, mock_hook, to_dict_mock): op = ListModelsOperator( task_id=TASK_ID, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, region=GCP_LOCATION, project_id=GCP_PROJECT, @@ -1548,9 +1450,7 @@ def test_execute(self, mock_hook, to_dict_mock): metadata=METADATA, ) op.execute(context={"ti": mock.MagicMock()}) - mock_hook.assert_called_once_with( - gcp_conn_id=GCP_CONN_ID, delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN - ) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) mock_hook.return_value.list_models.assert_called_once_with( region=GCP_LOCATION, project_id=GCP_PROJECT, @@ -1572,7 +1472,6 @@ def test_execute(self, mock_hook, to_dict_mock): op = UploadModelOperator( task_id=TASK_ID, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, region=GCP_LOCATION, project_id=GCP_PROJECT, @@ -1582,9 +1481,7 @@ def test_execute(self, mock_hook, to_dict_mock): metadata=METADATA, ) op.execute(context={"ti": mock.MagicMock()}) - mock_hook.assert_called_once_with( - gcp_conn_id=GCP_CONN_ID, delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN - ) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) mock_hook.return_value.upload_model.assert_called_once_with( region=GCP_LOCATION, project_id=GCP_PROJECT, diff --git a/tests/providers/google/cloud/sensors/test_bigquery.py b/tests/providers/google/cloud/sensors/test_bigquery.py index 1482f5bfd6e77..b699f7579773f 100644 --- a/tests/providers/google/cloud/sensors/test_bigquery.py +++ b/tests/providers/google/cloud/sensors/test_bigquery.py @@ -35,7 +35,6 @@ TEST_PROJECT_ID = "test_project" TEST_DATASET_ID = "test_dataset" TEST_TABLE_ID = "test_table" -TEST_DELEGATE_TO = "test_delegate_to" TEST_GCP_CONN_ID = "test_gcp_conn_id" TEST_PARTITION_ID = "20200101" TEST_IMPERSONATION_CHAIN = ["ACCOUNT_1", "ACCOUNT_2", "ACCOUNT_3"] @@ -50,7 +49,6 @@ def test_passing_arguments_to_hook(self, mock_hook): dataset_id=TEST_DATASET_ID, table_id=TEST_TABLE_ID, gcp_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) mock_hook.return_value.table_exists.return_value = True @@ -60,7 +58,6 @@ def test_passing_arguments_to_hook(self, mock_hook): mock_hook.assert_called_once_with( gcp_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) mock_hook.return_value.table_exists.assert_called_once_with( @@ -133,7 +130,6 @@ def test_passing_arguments_to_hook(self, mock_hook): table_id=TEST_TABLE_ID, partition_id=TEST_PARTITION_ID, gcp_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) mock_hook.return_value.table_partition_exists.return_value = True @@ -143,7 +139,6 @@ def test_passing_arguments_to_hook(self, mock_hook): mock_hook.assert_called_once_with( gcp_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) mock_hook.return_value.table_partition_exists.assert_called_once_with( diff --git a/tests/providers/google/cloud/sensors/test_dataflow.py b/tests/providers/google/cloud/sensors/test_dataflow.py index 41c6d275d5e6b..36d8840c815f2 100644 --- a/tests/providers/google/cloud/sensors/test_dataflow.py +++ b/tests/providers/google/cloud/sensors/test_dataflow.py @@ -34,7 +34,6 @@ TEST_JOB_ID = "test_job_id" TEST_PROJECT_ID = "test_project" TEST_LOCATION = "us-central1" -TEST_DELEGATE_TO = "test_delegate_to" TEST_GCP_CONN_ID = "test_gcp_conn_id" TEST_IMPERSONATION_CHAIN = ["ACCOUNT_1", "ACCOUNT_2", "ACCOUNT_3"] @@ -57,7 +56,6 @@ def test_poke(self, mock_hook, expected_status, current_status, sensor_return): location=TEST_LOCATION, project_id=TEST_PROJECT_ID, gcp_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) mock_get_job.return_value = {"id": TEST_JOB_ID, "currentState": current_status} @@ -67,7 +65,6 @@ def test_poke(self, mock_hook, expected_status, current_status, sensor_return): mock_hook.assert_called_once_with( gcp_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) mock_get_job.assert_called_once_with( @@ -84,7 +81,6 @@ def test_poke_raise_exception(self, mock_hook): location=TEST_LOCATION, project_id=TEST_PROJECT_ID, gcp_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) mock_get_job.return_value = {"id": TEST_JOB_ID, "currentState": DataflowJobStatus.JOB_STATE_CANCELLED} @@ -98,7 +94,6 @@ def test_poke_raise_exception(self, mock_hook): mock_hook.assert_called_once_with( gcp_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) mock_get_job.assert_called_once_with( @@ -129,7 +124,6 @@ def test_poke(self, mock_hook, job_current_state, fail_on_terminal_state): location=TEST_LOCATION, project_id=TEST_PROJECT_ID, gcp_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) mock_get_job.return_value = {"id": TEST_JOB_ID, "currentState": job_current_state} @@ -139,7 +133,6 @@ def test_poke(self, mock_hook, job_current_state, fail_on_terminal_state): mock_hook.assert_called_once_with( gcp_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) mock_fetch_job_metrics_by_id.assert_called_once_with( @@ -172,7 +165,6 @@ def test_poke(self, mock_hook, job_current_state, fail_on_terminal_state): location=TEST_LOCATION, project_id=TEST_PROJECT_ID, gcp_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) mock_get_job.return_value = {"id": TEST_JOB_ID, "currentState": job_current_state} @@ -183,7 +175,6 @@ def test_poke(self, mock_hook, job_current_state, fail_on_terminal_state): mock_hook.assert_called_once_with( gcp_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) mock_fetch_job_messages_by_id.assert_called_once_with( @@ -205,7 +196,6 @@ def test_poke_raise_exception(self, mock_hook): location=TEST_LOCATION, project_id=TEST_PROJECT_ID, gcp_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) mock_get_job.return_value = {"id": TEST_JOB_ID, "currentState": DataflowJobStatus.JOB_STATE_DONE} @@ -219,7 +209,6 @@ def test_poke_raise_exception(self, mock_hook): mock_hook.assert_called_once_with( gcp_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) mock_fetch_job_messages_by_id.assert_not_called() @@ -249,7 +238,6 @@ def test_poke(self, mock_hook, job_current_state, fail_on_terminal_state): location=TEST_LOCATION, project_id=TEST_PROJECT_ID, gcp_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) mock_get_job.return_value = {"id": TEST_JOB_ID, "currentState": job_current_state} @@ -260,7 +248,6 @@ def test_poke(self, mock_hook, job_current_state, fail_on_terminal_state): mock_hook.assert_called_once_with( gcp_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) mock_fetch_job_autoscaling_events_by_id.assert_called_once_with( @@ -282,7 +269,6 @@ def test_poke_raise_exception_on_terminal_state(self, mock_hook): location=TEST_LOCATION, project_id=TEST_PROJECT_ID, gcp_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) mock_get_job.return_value = {"id": TEST_JOB_ID, "currentState": DataflowJobStatus.JOB_STATE_DONE} @@ -296,7 +282,6 @@ def test_poke_raise_exception_on_terminal_state(self, mock_hook): mock_hook.assert_called_once_with( gcp_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) mock_fetch_job_autoscaling_events_by_id.assert_not_called() diff --git a/tests/providers/google/cloud/sensors/test_datafusion.py b/tests/providers/google/cloud/sensors/test_datafusion.py index 36a791f30647f..32dcfbb0508b8 100644 --- a/tests/providers/google/cloud/sensors/test_datafusion.py +++ b/tests/providers/google/cloud/sensors/test_datafusion.py @@ -32,7 +32,6 @@ PIPELINE_ID = "test_pipeline_id" PROJECT_ID = "test_project_id" GCP_CONN_ID = "test_conn_id" -DELEGATE_TO = "test_delegate_to" IMPERSONATION_CHAIN = ["ACCOUNT_1", "ACCOUNT_2", "ACCOUNT_3"] FAILURE_STATUSES = {"FAILED"} @@ -58,7 +57,6 @@ def test_poke(self, mock_hook, expected_status, current_status, sensor_return): instance_name=INSTANCE_NAME, location=LOCATION, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, ) @@ -69,7 +67,6 @@ def test_poke(self, mock_hook, expected_status, current_status, sensor_return): mock_hook.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, ) @@ -91,7 +88,6 @@ def test_assertion(self, mock_hook): instance_name=INSTANCE_NAME, location=LOCATION, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, ) @@ -117,7 +113,6 @@ def test_not_found_exception(self, mock_hook): instance_name=INSTANCE_NAME, location=LOCATION, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, ) diff --git a/tests/providers/google/cloud/sensors/test_dataplex.py b/tests/providers/google/cloud/sensors/test_dataplex.py index 9871a2f4c5d39..bde6a61be1627 100644 --- a/tests/providers/google/cloud/sensors/test_dataplex.py +++ b/tests/providers/google/cloud/sensors/test_dataplex.py @@ -34,7 +34,6 @@ DATAPLEX_TASK_ID = "testTask001" GCP_CONN_ID = "google_cloud_default" -DELEGATE_TO = "test-delegate-to" API_VERSION = "v1" IMPERSONATION_CHAIN = ["ACCOUNT_1", "ACCOUNT_2", "ACCOUNT_3"] @@ -58,7 +57,6 @@ def test_done(self, mock_hook): dataplex_task_id=DATAPLEX_TASK_ID, api_version=API_VERSION, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, ) result = sensor.poke(context={}) @@ -87,7 +85,6 @@ def test_deleting(self, mock_hook): dataplex_task_id=DATAPLEX_TASK_ID, api_version=API_VERSION, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, ) diff --git a/tests/providers/google/cloud/sensors/test_gcs.py b/tests/providers/google/cloud/sensors/test_gcs.py index 2f4c6a40c7f5c..8a66fb0081169 100644 --- a/tests/providers/google/cloud/sensors/test_gcs.py +++ b/tests/providers/google/cloud/sensors/test_gcs.py @@ -40,8 +40,6 @@ TEST_OBJECT = "TEST_OBJECT" -TEST_DELEGATE_TO = "TEST_DELEGATE_TO" - TEST_GCP_CONN_ID = "TEST_GCP_CONN_ID" TEST_IMPERSONATION_CHAIN = ["ACCOUNT_1", "ACCOUNT_2", "ACCOUNT_3"] @@ -83,7 +81,6 @@ def test_should_pass_argument_to_hook(self, mock_hook): bucket=TEST_BUCKET, object=TEST_OBJECT, google_cloud_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) mock_hook.return_value.exists.return_value = True @@ -92,7 +89,6 @@ def test_should_pass_argument_to_hook(self, mock_hook): assert result is True mock_hook.assert_called_once_with( - delegate_to=TEST_DELEGATE_TO, gcp_conn_id=TEST_GCP_CONN_ID, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) @@ -216,14 +212,12 @@ def test_should_pass_argument_to_hook(self, mock_hook): bucket=TEST_BUCKET, object=TEST_OBJECT, google_cloud_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) mock_hook.return_value.is_updated_after.return_value = True result = task.poke(mock.MagicMock()) mock_hook.assert_called_once_with( - delegate_to=TEST_DELEGATE_TO, gcp_conn_id=TEST_GCP_CONN_ID, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) @@ -239,14 +233,12 @@ def test_should_pass_arguments_to_hook(self, mock_hook): bucket=TEST_BUCKET, prefix=TEST_PREFIX, google_cloud_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) mock_hook.return_value.list.return_value = ["NOT_EMPTY_LIST"] result = task.poke(mock.MagicMock) mock_hook.assert_called_once_with( - delegate_to=TEST_DELEGATE_TO, gcp_conn_id=TEST_GCP_CONN_ID, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) @@ -260,7 +252,6 @@ def test_should_return_false_on_empty_list(self, mock_hook): bucket=TEST_BUCKET, prefix=TEST_PREFIX, google_cloud_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, ) mock_hook.return_value.list.return_value = [] result = task.poke(mock.MagicMock) @@ -274,7 +265,6 @@ def test_execute(self, mock_hook): bucket=TEST_BUCKET, prefix=TEST_PREFIX, google_cloud_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, impersonation_chain=TEST_IMPERSONATION_CHAIN, poke_interval=0, ) @@ -284,7 +274,6 @@ def test_execute(self, mock_hook): response = task.execute(None) mock_hook.assert_called_once_with( - delegate_to=TEST_DELEGATE_TO, gcp_conn_id=TEST_GCP_CONN_ID, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) @@ -319,7 +308,6 @@ def setup_method(self): min_objects=1, allow_delete=False, google_cloud_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, impersonation_chain=TEST_IMPERSONATION_CHAIN, dag=self.dag, ) @@ -331,7 +319,6 @@ def test_get_gcs_hook(self, mock_hook): self.sensor._get_gcs_hook() mock_hook.assert_called_once_with( gcp_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) assert mock_hook.return_value == self.sensor.hook diff --git a/tests/providers/google/cloud/transfers/test_adls_to_gcs.py b/tests/providers/google/cloud/transfers/test_adls_to_gcs.py index 8575fc353f136..914cbdc672339 100644 --- a/tests/providers/google/cloud/transfers/test_adls_to_gcs.py +++ b/tests/providers/google/cloud/transfers/test_adls_to_gcs.py @@ -96,7 +96,6 @@ def test_execute(self, gcs_mock_hook, adls_one_mock_hook, adls_two_mock_hook): adls_two_mock_hook.assert_called_once_with(azure_data_lake_conn_id=AZURE_CONN_ID) gcs_mock_hook.assert_called_once_with( gcp_conn_id=GCS_CONN_ID, - delegate_to=None, impersonation_chain=IMPERSONATION_CHAIN, ) diff --git a/tests/providers/google/cloud/transfers/test_azure_fileshare_to_gcs.py b/tests/providers/google/cloud/transfers/test_azure_fileshare_to_gcs.py index 49bae9dfdf841..09d2162868891 100644 --- a/tests/providers/google/cloud/transfers/test_azure_fileshare_to_gcs.py +++ b/tests/providers/google/cloud/transfers/test_azure_fileshare_to_gcs.py @@ -84,7 +84,6 @@ def test_execute(self, gcs_mock_hook, azure_fileshare_mock_hook): gcs_mock_hook.assert_called_once_with( gcp_conn_id=GCS_CONN_ID, - delegate_to=None, impersonation_chain=IMPERSONATION_CHAIN, ) diff --git a/tests/providers/google/cloud/transfers/test_calendar_to_gcs.py b/tests/providers/google/cloud/transfers/test_calendar_to_gcs.py index 31c337780e5a0..9684fc4bc4d3d 100644 --- a/tests/providers/google/cloud/transfers/test_calendar_to_gcs.py +++ b/tests/providers/google/cloud/transfers/test_calendar_to_gcs.py @@ -71,7 +71,6 @@ def test_upload_data(self, mock_tempfile, mock_gcs_hook): # Test GCS Hook mock_gcs_hook.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, - delegate_to=None, impersonation_chain=IMPERSONATION_CHAIN, ) @@ -102,7 +101,6 @@ def test_execute(self, mock_upload_data, mock_calendar_hook): mock_calendar_hook.assert_called_once_with( api_version=API_VERSION, gcp_conn_id=GCP_CONN_ID, - delegate_to=None, impersonation_chain=IMPERSONATION_CHAIN, ) diff --git a/tests/providers/google/cloud/transfers/test_gcs_to_sftp.py b/tests/providers/google/cloud/transfers/test_gcs_to_sftp.py index f5c9b5b987f32..9c6884766bea2 100644 --- a/tests/providers/google/cloud/transfers/test_gcs_to_sftp.py +++ b/tests/providers/google/cloud/transfers/test_gcs_to_sftp.py @@ -29,7 +29,6 @@ TASK_ID = "test-gcs-to-sftp-operator" GCP_CONN_ID = "GCP_CONN_ID" SFTP_CONN_ID = "SFTP_CONN_ID" -DELEGATE_TO = "DELEGATE_TO" IMPERSONATION_CHAIN = ["ACCOUNT_1", "ACCOUNT_2", "ACCOUNT_3"] TEST_BUCKET = "test-bucket" DESTINATION_SFTP = "destination_path" @@ -59,13 +58,11 @@ def test_execute_copy_single_file( move_object=False, gcp_conn_id=GCP_CONN_ID, sftp_conn_id=SFTP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, ) task.execute({}) gcs_hook_mock.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, ) sftp_hook_mock.assert_called_once_with(SFTP_CONN_ID) @@ -103,13 +100,11 @@ def test_execute_move_single_file( move_object=True, gcp_conn_id=GCP_CONN_ID, sftp_conn_id=SFTP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, ) task.execute(None) gcs_hook_mock.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, ) sftp_hook_mock.assert_called_once_with(SFTP_CONN_ID) @@ -196,7 +191,6 @@ def test_execute_copy_with_wildcard( move_object=False, gcp_conn_id=GCP_CONN_ID, sftp_conn_id=SFTP_CONN_ID, - delegate_to=DELEGATE_TO, ) operator.execute(None) @@ -289,7 +283,6 @@ def test_execute_move_with_wildcard( move_object=True, gcp_conn_id=GCP_CONN_ID, sftp_conn_id=SFTP_CONN_ID, - delegate_to=DELEGATE_TO, ) operator.execute(None) @@ -323,7 +316,6 @@ def test_execute_more_than_one_wildcard_exception(self, sftp_hook_mock, gcs_hook move_object=False, gcp_conn_id=GCP_CONN_ID, sftp_conn_id=SFTP_CONN_ID, - delegate_to=DELEGATE_TO, ) with pytest.raises(AirflowException): operator.execute(None) diff --git a/tests/providers/google/cloud/transfers/test_gdrive_to_local.py b/tests/providers/google/cloud/transfers/test_gdrive_to_local.py index 3118f1bec174c..c1dac32deb635 100644 --- a/tests/providers/google/cloud/transfers/test_gdrive_to_local.py +++ b/tests/providers/google/cloud/transfers/test_gdrive_to_local.py @@ -43,9 +43,7 @@ def test_execute(self, hook_mock): hook_mock.return_value.get_file_id.return_value = meta op.execute(context=None) - hook_mock.assert_called_once_with( - delegate_to=None, gcp_conn_id=GCP_CONN_ID, impersonation_chain=None - ) + hook_mock.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=None) hook_mock.return_value.download_file.assert_called_once_with( file_id=meta["id"], file_handle=mock.ANY diff --git a/tests/providers/google/cloud/transfers/test_s3_to_gcs.py b/tests/providers/google/cloud/transfers/test_s3_to_gcs.py index 5bcf44e7ba834..9b7d86d280859 100644 --- a/tests/providers/google/cloud/transfers/test_s3_to_gcs.py +++ b/tests/providers/google/cloud/transfers/test_s3_to_gcs.py @@ -87,7 +87,6 @@ def test_execute(self, gcs_mock_hook, s3_one_mock_hook, s3_two_mock_hook): s3_two_mock_hook.assert_called_once_with(aws_conn_id=AWS_CONN_ID, verify=None) gcs_mock_hook.assert_called_once_with( gcp_conn_id=GCS_CONN_ID, - delegate_to=None, impersonation_chain=IMPERSONATION_CHAIN, ) @@ -116,7 +115,6 @@ def test_execute_with_gzip(self, gcs_mock_hook, s3_one_mock_hook, s3_two_mock_ho operator.execute(None) gcs_mock_hook.assert_called_once_with( gcp_conn_id=GCS_CONN_ID, - delegate_to=None, impersonation_chain=None, ) gcs_mock_hook.return_value.upload.assert_has_calls( diff --git a/tests/providers/google/cloud/transfers/test_sftp_to_gcs.py b/tests/providers/google/cloud/transfers/test_sftp_to_gcs.py index c22f1b59060a3..433961eb6e904 100644 --- a/tests/providers/google/cloud/transfers/test_sftp_to_gcs.py +++ b/tests/providers/google/cloud/transfers/test_sftp_to_gcs.py @@ -29,7 +29,6 @@ TASK_ID = "test-gcs-to-sftp-operator" GCP_CONN_ID = "GCP_CONN_ID" SFTP_CONN_ID = "SFTP_CONN_ID" -DELEGATE_TO = "DELEGATE_TO" IMPERSONATION_CHAIN = ["ACCOUNT_1", "ACCOUNT_2", "ACCOUNT_3"] DEFAULT_MIME_TYPE = "application/octet-stream" @@ -64,13 +63,11 @@ def test_execute_copy_single_file(self, sftp_hook, gcs_hook): move_object=False, gcp_conn_id=GCP_CONN_ID, sftp_conn_id=SFTP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, ) task.execute(None) gcs_hook.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, ) sftp_hook.assert_called_once_with(SFTP_CONN_ID) @@ -100,14 +97,12 @@ def test_execute_copy_single_file_with_compression(self, sftp_hook, gcs_hook): move_object=False, gcp_conn_id=GCP_CONN_ID, sftp_conn_id=SFTP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, gzip=True, ) task.execute(None) gcs_hook.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, ) sftp_hook.assert_called_once_with(SFTP_CONN_ID) @@ -137,13 +132,11 @@ def test_execute_move_single_file(self, sftp_hook, gcs_hook): move_object=True, gcp_conn_id=GCP_CONN_ID, sftp_conn_id=SFTP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, ) task.execute(None) gcs_hook.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, ) sftp_hook.assert_called_once_with(SFTP_CONN_ID) @@ -179,7 +172,6 @@ def test_execute_copy_with_wildcard(self, sftp_hook, gcs_hook): move_object=True, gcp_conn_id=GCP_CONN_ID, sftp_conn_id=SFTP_CONN_ID, - delegate_to=DELEGATE_TO, ) task.execute(None) @@ -231,7 +223,6 @@ def test_execute_move_with_wildcard(self, sftp_hook, gcs_hook): move_object=True, gcp_conn_id=GCP_CONN_ID, sftp_conn_id=SFTP_CONN_ID, - delegate_to=DELEGATE_TO, ) task.execute(None) @@ -253,7 +244,6 @@ def test_execute_more_than_one_wildcard_exception(self, sftp_hook, gcs_hook): move_object=False, gcp_conn_id=GCP_CONN_ID, sftp_conn_id=SFTP_CONN_ID, - delegate_to=DELEGATE_TO, ) with pytest.raises(AirflowException) as ctx: task.execute(None) diff --git a/tests/providers/google/cloud/transfers/test_sheets_to_gcs.py b/tests/providers/google/cloud/transfers/test_sheets_to_gcs.py index 38573c183bfcc..797203ae818a6 100644 --- a/tests/providers/google/cloud/transfers/test_sheets_to_gcs.py +++ b/tests/providers/google/cloud/transfers/test_sheets_to_gcs.py @@ -101,12 +101,10 @@ def test_execute(self, mock_upload_data, mock_xcom, mock_sheet_hook, mock_gcs_ho mock_sheet_hook.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, - delegate_to=None, impersonation_chain=IMPERSONATION_CHAIN, ) mock_gcs_hook.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, - delegate_to=None, impersonation_chain=IMPERSONATION_CHAIN, ) diff --git a/tests/providers/google/cloud/triggers/test_bigquery_dts.py b/tests/providers/google/cloud/triggers/test_bigquery_dts.py index 866e4756cf073..90821ea7c579e 100644 --- a/tests/providers/google/cloud/triggers/test_bigquery_dts.py +++ b/tests/providers/google/cloud/triggers/test_bigquery_dts.py @@ -32,7 +32,6 @@ RUN_ID = "test-run-id" POLL_INTERVAL = 10 GCP_CONN_ID = "google-cloud-default-id" -DELEGATE_TO = "test-delegate-to" LOCATION = "us-central1" IMPERSONATION_CHAIN = ["test", "chain"] @@ -45,7 +44,6 @@ def trigger(): run_id=RUN_ID, poll_interval=POLL_INTERVAL, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, location=LOCATION, impersonation_chain=IMPERSONATION_CHAIN, ) @@ -65,7 +63,6 @@ def test_serialize(self, trigger): "run_id": RUN_ID, "poll_interval": POLL_INTERVAL, "gcp_conn_id": GCP_CONN_ID, - "delegate_to": DELEGATE_TO, "location": LOCATION, "impersonation_chain": IMPERSONATION_CHAIN, } @@ -74,7 +71,6 @@ def test_serialize(self, trigger): "attr, expected_value", [ ("gcp_conn_id", GCP_CONN_ID), - ("delegate_to", DELEGATE_TO), ("location", LOCATION), ("impersonation_chain", IMPERSONATION_CHAIN), ], diff --git a/tests/providers/google/cloud/triggers/test_cloud_build.py b/tests/providers/google/cloud/triggers/test_cloud_build.py index 2fef12d9efe44..8687263f8e65e 100644 --- a/tests/providers/google/cloud/triggers/test_cloud_build.py +++ b/tests/providers/google/cloud/triggers/test_cloud_build.py @@ -86,7 +86,6 @@ def trigger(): project_id=TEST_PROJECT_ID, gcp_conn_id=TEST_CONN_ID, impersonation_chain=None, - delegate_to=None, poll_interval=TEST_POLL_INTERVAL, location=TEST_LOCATION, ) @@ -112,7 +111,6 @@ def test_serialization(self, trigger): "project_id": TEST_PROJECT_ID, "gcp_conn_id": TEST_CONN_ID, "impersonation_chain": None, - "delegate_to": None, "poll_interval": TEST_POLL_INTERVAL, "location": TEST_LOCATION, } diff --git a/tests/providers/google/cloud/triggers/test_dataflow.py b/tests/providers/google/cloud/triggers/test_dataflow.py index 04580443baa05..6da1a1e79484a 100644 --- a/tests/providers/google/cloud/triggers/test_dataflow.py +++ b/tests/providers/google/cloud/triggers/test_dataflow.py @@ -33,7 +33,6 @@ JOB_ID = "test_job_id_2012-12-23-10:00" LOCATION = "us-central1" GCP_CONN_ID = "test_gcp_conn_id" -DELEGATE_TO = "delegating_to_something" POLL_SLEEP = 20 IMPERSONATION_CHAIN = ["impersonate", "this"] CANCEL_TIMEOUT = 10 * 420 @@ -46,7 +45,6 @@ def trigger(): job_id=JOB_ID, location=LOCATION, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, poll_sleep=POLL_SLEEP, impersonation_chain=IMPERSONATION_CHAIN, cancel_timeout=CANCEL_TIMEOUT, @@ -77,7 +75,6 @@ def test_serialize(self, trigger): "job_id": JOB_ID, "location": LOCATION, "gcp_conn_id": GCP_CONN_ID, - "delegate_to": DELEGATE_TO, "poll_sleep": POLL_SLEEP, "impersonation_chain": IMPERSONATION_CHAIN, "cancel_timeout": CANCEL_TIMEOUT, @@ -89,7 +86,6 @@ def test_serialize(self, trigger): "attr, expected", [ ("gcp_conn_id", GCP_CONN_ID), - ("delegate_to", DELEGATE_TO), ("poll_sleep", POLL_SLEEP), ("impersonation_chain", IMPERSONATION_CHAIN), ("cancel_timeout", CANCEL_TIMEOUT), diff --git a/tests/providers/google/cloud/triggers/test_dataproc.py b/tests/providers/google/cloud/triggers/test_dataproc.py index a92328ea8d00b..73da7c077c930 100644 --- a/tests/providers/google/cloud/triggers/test_dataproc.py +++ b/tests/providers/google/cloud/triggers/test_dataproc.py @@ -296,7 +296,6 @@ def test_async_cluster_trigger_serialization_should_execute_successfully(self, w "project_id": TEST_PROJECT_ID, "region": TEST_REGION, "gcp_conn_id": TEST_GCP_CONN_ID, - "delegate_to": None, "impersonation_chain": None, "polling_interval_seconds": TEST_POLL_INTERVAL, } diff --git a/tests/providers/google/cloud/triggers/test_kubernetes_engine.py b/tests/providers/google/cloud/triggers/test_kubernetes_engine.py index eb3497012f97b..3f78fb3a049f3 100644 --- a/tests/providers/google/cloud/triggers/test_kubernetes_engine.py +++ b/tests/providers/google/cloud/triggers/test_kubernetes_engine.py @@ -58,7 +58,6 @@ PROJECT_ID = "test-project-id" LOCATION = "us-central1-c" GCP_CONN_ID = "test-non-existing-project-id" -DELEGATE_TO = "test-delegate-to" IMPERSONATION_CHAIN = ["impersonate", "this", "test"] TRIGGER_PATH = "airflow.providers.google.cloud.triggers.kubernetes_engine.GKEOperationTrigger" EXC_MSG = "test error msg" @@ -302,7 +301,6 @@ def operation_trigger(): project_id=PROJECT_ID, location=LOCATION, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, poll_interval=POLL_INTERVAL, ) @@ -329,7 +327,6 @@ def test_serialize(self, operation_trigger): "project_id": PROJECT_ID, "location": LOCATION, "gcp_conn_id": GCP_CONN_ID, - "delegate_to": DELEGATE_TO, "impersonation_chain": IMPERSONATION_CHAIN, "poll_interval": POLL_INTERVAL, } diff --git a/tests/providers/google/cloud/utils/base_gcp_mock.py b/tests/providers/google/cloud/utils/base_gcp_mock.py index 77a02cf95d2ab..1836e7365c22d 100644 --- a/tests/providers/google/cloud/utils/base_gcp_mock.py +++ b/tests/providers/google/cloud/utils/base_gcp_mock.py @@ -28,33 +28,33 @@ def mock_base_gcp_hook_default_project_id( self, gcp_conn_id="google_cloud_default", - delegate_to=None, impersonation_chain=None, + delegate_to=None, ): self.extras_list = {"project": GCP_PROJECT_ID_HOOK_UNIT_TEST} self._conn = gcp_conn_id - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain self._client = None self._conn = None self._cached_credentials = None self._cached_project_id = None + self.delegate_to = delegate_to def mock_base_gcp_hook_no_default_project_id( self, gcp_conn_id="google_cloud_default", - delegate_to=None, impersonation_chain=None, + delegate_to=None, ): self.extras_list = {} self._conn = gcp_conn_id - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain self._client = None self._conn = None self._cached_credentials = None self._cached_project_id = None + self.delegate_to = delegate_to GCP_CONNECTION_WITH_PROJECT_ID = Connection(extra=json.dumps({"project": GCP_PROJECT_ID_HOOK_UNIT_TEST})) diff --git a/tests/providers/google/suite/transfers/test_gcs_to_gdrive.py b/tests/providers/google/suite/transfers/test_gcs_to_gdrive.py index 9721543cc790b..5730a4b66ed52 100644 --- a/tests/providers/google/suite/transfers/test_gcs_to_gdrive.py +++ b/tests/providers/google/suite/transfers/test_gcs_to_gdrive.py @@ -48,7 +48,6 @@ def test_should_copy_single_file(self, mock_named_temporary_file, mock_gdrive, m mock_gcs_hook.assert_has_calls( [ mock.call( - delegate_to=None, gcp_conn_id="google_cloud_default", impersonation_chain=None, ), @@ -61,7 +60,6 @@ def test_should_copy_single_file(self, mock_named_temporary_file, mock_gdrive, m mock_gdrive.assert_has_calls( [ mock.call( - delegate_to=None, gcp_conn_id="google_cloud_default", impersonation_chain=None, ), @@ -93,7 +91,6 @@ def test_should_copy_files(self, mock_named_temporary_file, mock_gdrive, mock_gc mock_gcs_hook.assert_has_calls( [ mock.call( - delegate_to=None, gcp_conn_id="google_cloud_default", impersonation_chain=IMPERSONATION_CHAIN, ), @@ -107,7 +104,6 @@ def test_should_copy_files(self, mock_named_temporary_file, mock_gdrive, mock_gc mock_gdrive.assert_has_calls( [ mock.call( - delegate_to=None, gcp_conn_id="google_cloud_default", impersonation_chain=IMPERSONATION_CHAIN, ), @@ -137,7 +133,6 @@ def test_should_move_files(self, mock_named_temporary_file, mock_gdrive, mock_gc mock_gcs_hook.assert_has_calls( [ mock.call( - delegate_to=None, gcp_conn_id="google_cloud_default", impersonation_chain=IMPERSONATION_CHAIN, ), @@ -154,7 +149,6 @@ def test_should_move_files(self, mock_named_temporary_file, mock_gdrive, mock_gc mock_gdrive.assert_has_calls( [ mock.call( - delegate_to=None, gcp_conn_id="google_cloud_default", impersonation_chain=IMPERSONATION_CHAIN, ), diff --git a/tests/providers/google/suite/transfers/test_gcs_to_sheets.py b/tests/providers/google/suite/transfers/test_gcs_to_sheets.py index 79313be6a5c27..30f48d5137372 100644 --- a/tests/providers/google/suite/transfers/test_gcs_to_sheets.py +++ b/tests/providers/google/suite/transfers/test_gcs_to_sheets.py @@ -52,12 +52,10 @@ def test_execute(self, mock_reader, mock_tempfile, mock_sheet_hook, mock_gcs_hoo mock_sheet_hook.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, - delegate_to=None, impersonation_chain=IMPERSONATION_CHAIN, ) mock_gcs_hook.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, - delegate_to=None, impersonation_chain=IMPERSONATION_CHAIN, ) diff --git a/tests/providers/microsoft/azure/transfers/test_azure_blob_to_gcs.py b/tests/providers/microsoft/azure/transfers/test_azure_blob_to_gcs.py index 0bb5b9535d13f..e7d2ba6833814 100644 --- a/tests/providers/microsoft/azure/transfers/test_azure_blob_to_gcs.py +++ b/tests/providers/microsoft/azure/transfers/test_azure_blob_to_gcs.py @@ -29,7 +29,6 @@ OBJECT_NAME = "file.txt" FILENAME = "file.txt" GZIP = False -DELEGATE_TO = None IMPERSONATION_CHAIN = None TASK_ID = "transfer_file" @@ -46,7 +45,6 @@ def test_init(self): object_name=OBJECT_NAME, filename=FILENAME, gzip=GZIP, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, task_id=TASK_ID, ) @@ -59,7 +57,6 @@ def test_init(self): assert operator.object_name == OBJECT_NAME assert operator.filename == FILENAME assert operator.gzip == GZIP - assert operator.delegate_to == DELEGATE_TO assert operator.impersonation_chain == IMPERSONATION_CHAIN assert operator.task_id == TASK_ID @@ -77,7 +74,6 @@ def test_execute(self, mock_temp, mock_hook_gcs, mock_hook_wasb): object_name=OBJECT_NAME, filename=FILENAME, gzip=GZIP, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, task_id=TASK_ID, ) @@ -91,7 +87,7 @@ def test_execute(self, mock_temp, mock_hook_gcs, mock_hook_wasb): blob_name=BLOB_NAME, ) mock_hook_gcs.assert_called_once_with( - gcp_conn_id=GCP_CONN_ID, delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN + gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN ) mock_hook_gcs.return_value.upload.assert_called_once_with( bucket_name=BUCKET_NAME, diff --git a/tests/providers/presto/transfers/test_gcs_presto.py b/tests/providers/presto/transfers/test_gcs_presto.py index fe0689099bbae..fae3fa0579764 100644 --- a/tests/providers/presto/transfers/test_gcs_presto.py +++ b/tests/providers/presto/transfers/test_gcs_presto.py @@ -55,7 +55,6 @@ def test_execute_without_schema(self, mock_tempfile, mock_gcs_hook, mock_presto_ mock_gcs_hook.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, - delegate_to=None, impersonation_chain=IMPERSONATION_CHAIN, ) @@ -92,7 +91,6 @@ def test_execute_schema_fields(self, mock_tempfile, mock_gcs_hook, mock_presto_h mock_gcs_hook.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, - delegate_to=None, impersonation_chain=IMPERSONATION_CHAIN, ) @@ -131,7 +129,6 @@ def test_execute_schema_json(self, mock_tempfile, mock_gcs_hook, mock_presto_hoo mock_gcs_hook.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, - delegate_to=None, impersonation_chain=IMPERSONATION_CHAIN, ) diff --git a/tests/providers/trino/transfers/test_gcs_trino.py b/tests/providers/trino/transfers/test_gcs_trino.py index ce5d2f8d10aea..90fbd21808ccd 100644 --- a/tests/providers/trino/transfers/test_gcs_trino.py +++ b/tests/providers/trino/transfers/test_gcs_trino.py @@ -55,7 +55,6 @@ def test_execute_without_schema(self, mock_tempfile, mock_gcs_hook, mock_trino_h mock_gcs_hook.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, - delegate_to=None, impersonation_chain=IMPERSONATION_CHAIN, ) @@ -92,7 +91,6 @@ def test_execute_schema_fields(self, mock_tempfile, mock_gcs_hook, mock_trino_ho mock_gcs_hook.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, - delegate_to=None, impersonation_chain=IMPERSONATION_CHAIN, ) @@ -131,7 +129,6 @@ def test_execute_schema_json(self, mock_tempfile, mock_gcs_hook, mock_trino_hook mock_gcs_hook.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, - delegate_to=None, impersonation_chain=IMPERSONATION_CHAIN, ) diff --git a/tests/system/providers/microsoft/azure/example_azure_blob_to_gcs.py b/tests/system/providers/microsoft/azure/example_azure_blob_to_gcs.py index 8bd5de0d939c2..e05d74b1028df 100644 --- a/tests/system/providers/microsoft/azure/example_azure_blob_to_gcs.py +++ b/tests/system/providers/microsoft/azure/example_azure_blob_to_gcs.py @@ -57,7 +57,6 @@ object_name=GCP_OBJECT_NAME, filename=GCP_BUCKET_FILE_PATH, gzip=False, - delegate_to=None, impersonation_chain=None, ) # [END how_to_azure_blob_to_gcs]