You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by ka...@apache.org on 2020/08/24 11:48:49 UTC

[airflow] branch master updated: Implement impersonation in google operators (#10052)

This is an automated email from the ASF dual-hosted git repository.

kamilbregula pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/airflow.git


The following commit(s) were added to refs/heads/master by this push:
     new 3734876  Implement impersonation in google operators (#10052)
3734876 is described below

commit 3734876d9898067ee933b84af522d53df6160d7f
Author: Kamil Olszewski <34...@users.noreply.github.com>
AuthorDate: Mon Aug 24 13:47:59 2020 +0200

    Implement impersonation in google operators (#10052)
    
    Co-authored-by: Kamil Olszewski <ka...@polidea.com>
---
 UPDATING.md                                        |   6 +
 .../providers/amazon/aws/transfers/gcs_to_s3.py    |  59 ++-
 .../amazon/aws/transfers/google_api_to_s3.py       |  20 +-
 airflow/providers/google/ads/operators/ads.py      |  21 +-
 .../providers/google/ads/transfers/ads_to_gcs.py   |  20 +-
 airflow/providers/google/cloud/hooks/bigquery.py   |   4 +-
 airflow/providers/google/cloud/hooks/dataflow.py   |   2 +-
 airflow/providers/google/cloud/hooks/datastore.py  |   4 +-
 airflow/providers/google/cloud/hooks/gcs.py        |   2 +-
 .../google/cloud/hooks/kubernetes_engine.py        |   2 +-
 airflow/providers/google/cloud/operators/automl.py | 233 ++++++++-
 .../providers/google/cloud/operators/bigquery.py   | 320 ++++++++++--
 .../google/cloud/operators/bigquery_dts.py         |  54 +-
 .../providers/google/cloud/operators/bigtable.py   | 114 ++++-
 .../google/cloud/operators/cloud_build.py          |  21 +-
 .../google/cloud/operators/cloud_memorystore.py    | 193 ++++++-
 .../providers/google/cloud/operators/cloud_sql.py  | 155 +++++-
 .../operators/cloud_storage_transfer_service.py    | 207 +++++++-
 .../providers/google/cloud/operators/compute.py    | 112 ++++-
 .../google/cloud/operators/datacatalog.py          | 359 ++++++++++++-
 .../providers/google/cloud/operators/dataflow.py   |  33 +-
 .../providers/google/cloud/operators/datafusion.py | 192 +++++--
 .../providers/google/cloud/operators/dataproc.py   | 168 +++++--
 .../providers/google/cloud/operators/datastore.py  | 226 +++++++--
 airflow/providers/google/cloud/operators/dlp.py    | 557 +++++++++++++++++++--
 .../providers/google/cloud/operators/functions.py  |  62 ++-
 airflow/providers/google/cloud/operators/gcs.py    | 151 +++++-
 .../google/cloud/operators/kubernetes_engine.py    |  42 +-
 .../google/cloud/operators/life_sciences.py        |  21 +-
 .../providers/google/cloud/operators/mlengine.py   | 266 ++++++++--
 .../google/cloud/operators/natural_language.py     |  74 ++-
 airflow/providers/google/cloud/operators/pubsub.py | 153 ++++--
 .../providers/google/cloud/operators/spanner.py    | 111 +++-
 .../google/cloud/operators/speech_to_text.py       |  21 +-
 .../google/cloud/operators/stackdriver.py          | 232 +++++++--
 airflow/providers/google/cloud/operators/tasks.py  | 246 ++++++++-
 .../google/cloud/operators/text_to_speech.py       |  24 +-
 .../providers/google/cloud/operators/translate.py  |  22 +-
 .../google/cloud/operators/translate_speech.py     |  27 +-
 .../google/cloud/operators/video_intelligence.py   |  56 ++-
 airflow/providers/google/cloud/operators/vision.py | 323 ++++++++++--
 airflow/providers/google/cloud/sensors/bigquery.py |  43 +-
 .../providers/google/cloud/sensors/bigquery_dts.py |  18 +-
 airflow/providers/google/cloud/sensors/bigtable.py |  20 +-
 .../sensors/cloud_storage_transfer_service.py      |  20 +-
 airflow/providers/google/cloud/sensors/gcs.py      |  87 +++-
 airflow/providers/google/cloud/sensors/pubsub.py   |  22 +-
 .../google/cloud/transfers/adls_to_gcs.py          |  23 +-
 .../google/cloud/transfers/bigquery_to_bigquery.py |  24 +-
 .../google/cloud/transfers/bigquery_to_gcs.py      |  24 +-
 .../google/cloud/transfers/bigquery_to_mysql.py    |  25 +-
 .../google/cloud/transfers/cassandra_to_gcs.py     |  25 +-
 .../google/cloud/transfers/facebook_ads_to_gcs.py  |  20 +-
 .../google/cloud/transfers/gcs_to_bigquery.py      |  27 +-
 .../providers/google/cloud/transfers/gcs_to_gcs.py |  21 +-
 .../google/cloud/transfers/gcs_to_local.py         |  22 +-
 .../google/cloud/transfers/gcs_to_sftp.py          |  23 +-
 .../google/cloud/transfers/local_to_gcs.py         |  18 +-
 .../providers/google/cloud/transfers/s3_to_gcs.py  |  24 +-
 .../google/cloud/transfers/sftp_to_gcs.py          |  24 +-
 .../google/cloud/transfers/sheets_to_gcs.py        |  30 +-
 .../providers/google/cloud/transfers/sql_to_gcs.py |  26 +-
 .../google/cloud/utils/credentials_provider.py     |  19 +
 .../google/firebase/operators/firestore.py         |  21 +-
 .../marketing_platform/operators/analytics.py      | 117 ++++-
 .../operators/campaign_manager.py                  | 118 ++++-
 .../marketing_platform/operators/display_video.py  | 147 +++++-
 .../marketing_platform/operators/search_ads.py     |  44 +-
 .../marketing_platform/sensors/campaign_manager.py |  21 +-
 .../marketing_platform/sensors/display_video.py    |  40 +-
 .../marketing_platform/sensors/search_ads.py       |  25 +-
 airflow/providers/google/suite/operators/sheets.py |  25 +-
 .../google/suite/transfers/gcs_to_gdrive.py        |  31 +-
 .../google/suite/transfers/gcs_to_sheets.py        |  28 +-
 docs/howto/connection/gcp.rst                      |  96 ++++
 tests/providers/google/ads/operators/test_ads.py   |   7 +-
 .../google/ads/transfers/test_ads_to_gcs.py        |   9 +-
 .../google/cloud/operators/test_bigtable.py        | 233 ++++++---
 .../cloud/operators/test_cloud_memorystore.py      |  61 ++-
 .../google/cloud/operators/test_cloud_sql.py       |  72 ++-
 .../test_cloud_storage_transfer_service.py         | 109 +++-
 .../google/cloud/operators/test_compute.py         |  68 ++-
 .../google/cloud/operators/test_datacatalog.py     | 133 ++++-
 .../google/cloud/operators/test_dataproc.py        |  67 ++-
 .../google/cloud/operators/test_datastore.py       |  18 +-
 tests/providers/google/cloud/operators/test_dlp.py | 150 ++++--
 .../google/cloud/operators/test_functions.py       |  58 ++-
 tests/providers/google/cloud/operators/test_gcs.py |   5 +-
 .../google/cloud/operators/test_mlengine.py        | 121 ++++-
 .../google/cloud/operators/test_mlengine_utils.py  |   3 +-
 .../google/cloud/operators/test_spanner.py         | 105 +++-
 .../google/cloud/operators/test_speech_to_text.py  |  13 +-
 .../providers/google/cloud/operators/test_tasks.py |  65 ++-
 .../google/cloud/operators/test_text_to_speech.py  |  12 +-
 .../google/cloud/operators/test_translate.py       |   7 +-
 .../cloud/operators/test_translate_speech.py       |  22 +-
 .../cloud/operators/test_video_intelligence.py     |  31 +-
 .../google/cloud/operators/test_vision.py          | 100 +++-
 .../google/cloud/sensors/test_bigquery.py          |  13 +-
 .../google/cloud/sensors/test_bigtable.py          |  36 +-
 tests/providers/google/cloud/sensors/test_gcs.py   |  20 +-
 .../google/cloud/transfers/test_adls_to_gcs.py     |   9 +-
 .../cloud/transfers/test_facebook_ads_to_gcs.py    |   9 +-
 .../google/cloud/transfers/test_gcs_to_sftp.py     |  11 +-
 .../google/cloud/transfers/test_presto_to_gcs.py   |  24 +-
 .../google/cloud/transfers/test_s3_to_gcs.py       |  20 +-
 .../google/cloud/transfers/test_sftp_to_gcs.py     |  11 +-
 .../google/cloud/transfers/test_sheets_to_gcs.py   |  12 +-
 .../cloud/utils/test_credentials_provider.py       |  52 +-
 .../marketing_platform/operators/test_analytics.py |  40 +-
 .../operators/test_campaign_manager.py             |  24 +-
 .../operators/test_display_video.py                |  59 ++-
 .../operators/test_search_ads.py                   |  10 +-
 .../sensors/test_campaign_manager.py               |   5 +-
 .../sensors/test_display_video.py                  |  10 +-
 .../marketing_platform/sensors/test_search_ads.py  |   5 +-
 .../google/suite/transfers/test_gcs_to_gdrive.py   |  39 +-
 .../google/suite/transfers/test_gcs_to_sheets.py   |  12 +-
 118 files changed, 6845 insertions(+), 1258 deletions(-)

diff --git a/UPDATING.md b/UPDATING.md
index 667e655..1fa9bd9 100644
--- a/UPDATING.md
+++ b/UPDATING.md
@@ -1038,6 +1038,12 @@ of this provider.
 This section describes the changes that have been made, and what you need to do to update your if
 you use operators or hooks which integrate with Google services (including Google Cloud Platform - GCP).
 
+#### Direct impersonation added to operators communicating with Google services
+[Directly impersonating a service account](https://cloud.google.com/iam/docs/understanding-service-accounts#directly_impersonating_a_service_account)
+has been made possible for operators communicating with Google services via new argument called `impersonation_chain`
+(`google_impersonation_chain` in case of operators that also communicate with services of other cloud providers).
+As a result, GCSToS3Operator no longer derivatives from GCSListObjectsOperator.
+
 #### Normalize gcp_conn_id for Google Cloud Platform
 
 Previously not all hooks and operators related to Google Cloud Platform use
diff --git a/airflow/providers/amazon/aws/transfers/gcs_to_s3.py b/airflow/providers/amazon/aws/transfers/gcs_to_s3.py
index f7897a1..29b04dc 100644
--- a/airflow/providers/amazon/aws/transfers/gcs_to_s3.py
+++ b/airflow/providers/amazon/aws/transfers/gcs_to_s3.py
@@ -19,15 +19,15 @@
 This module contains Google Cloud Storage to S3 operator.
 """
 import warnings
-from typing import Iterable
+from typing import Iterable, Optional, Sequence, Union
 
+from airflow.models import BaseOperator
 from airflow.providers.amazon.aws.hooks.s3 import S3Hook
 from airflow.providers.google.cloud.hooks.gcs import GCSHook
-from airflow.providers.google.cloud.operators.gcs import GCSListObjectsOperator
 from airflow.utils.decorators import apply_defaults
 
 
-class GCSToS3Operator(GCSListObjectsOperator):
+class GCSToS3Operator(BaseOperator):
     """
     Synchronizes a Google Cloud Storage bucket with an S3 bucket.
 
@@ -45,8 +45,8 @@ class GCSToS3Operator(GCSListObjectsOperator):
     :param google_cloud_storage_conn_id: (Deprecated) The connection ID used to connect to Google Cloud
         Platform. This parameter has been deprecated. You should pass the gcp_conn_id parameter instead.
     :type google_cloud_storage_conn_id: str
-    :param delegate_to: The account to impersonate, if any.
-        For this to work, the service account making the request must have
+    :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.
     :type delegate_to: str
     :param dest_aws_conn_id: The destination S3 connection
@@ -73,8 +73,18 @@ class GCSToS3Operator(GCSListObjectsOperator):
         If set to False, will upload only the files that are in the origin but not
         in the destination bucket.
     :type replace: bool
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type google_impersonation_chain: Union[str, Sequence[str]]
     """
-    template_fields: Iterable[str] = ('bucket', 'prefix', 'delimiter', 'dest_s3_key')
+    template_fields: Iterable[str] = ('bucket', 'prefix', 'delimiter', 'dest_s3_key',
+                                      'google_impersonation_chain',)
     ui_color = '#f0eee4'
 
     @apply_defaults
@@ -89,7 +99,9 @@ class GCSToS3Operator(GCSListObjectsOperator):
                  dest_s3_key=None,
                  dest_verify=None,
                  replace=False,
+                 google_impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs):
+        super().__init__(**kwargs)
 
         if google_cloud_storage_conn_id:
             warnings.warn(
@@ -97,23 +109,32 @@ class GCSToS3Operator(GCSListObjectsOperator):
                 "the gcp_conn_id parameter.", DeprecationWarning, stacklevel=3)
             gcp_conn_id = google_cloud_storage_conn_id
 
-        super().__init__(
-            bucket=bucket,
-            prefix=prefix,
-            delimiter=delimiter,
-            gcp_conn_id=gcp_conn_id,
-            delegate_to=delegate_to,
-            **kwargs
-        )
-
+        self.bucket = bucket
+        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
         self.replace = replace
+        self.google_impersonation_chain = google_impersonation_chain
 
     def execute(self, context):
-        # use the super to list all files in an Google Cloud Storage bucket
-        files = super().execute(context)
+        # list all files in an Google Cloud Storage bucket
+        hook = GCSHook(
+            google_cloud_storage_conn_id=self.gcp_conn_id,
+            delegate_to=self.delegate_to,
+            impersonation_chain=self.google_impersonation_chain,
+        )
+
+        self.log.info('Getting list of the files. Bucket: %s; Delimiter: %s; Prefix: %s',
+                      self.bucket, self.delimiter, self.prefix)
+
+        files = hook.list(bucket_name=self.bucket,
+                          prefix=self.prefix,
+                          delimiter=self.delimiter)
+
         s3_hook = S3Hook(aws_conn_id=self.dest_aws_conn_id, verify=self.dest_verify)
 
         if not self.replace:
@@ -131,10 +152,6 @@ class GCSToS3Operator(GCSListObjectsOperator):
             files = list(set(files) - set(existing_files))
 
         if files:
-            hook = GCSHook(
-                google_cloud_storage_conn_id=self.gcp_conn_id,
-                delegate_to=self.delegate_to
-            )
 
             for file in files:
                 file_bytes = hook.download(self.bucket, file)
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 741b8a0..85695de 100644
--- a/airflow/providers/amazon/aws/transfers/google_api_to_s3.py
+++ b/airflow/providers/amazon/aws/transfers/google_api_to_s3.py
@@ -21,6 +21,7 @@ This module allows you to transfer data from any Google API endpoint into a S3 B
 """
 import json
 import sys
+from typing import Optional, Sequence, Union
 
 from airflow.models import BaseOperator
 from airflow.models.xcom import MAX_XCOM_SIZE
@@ -68,17 +69,27 @@ class GoogleApiToS3Operator(BaseOperator):
     :type s3_overwrite: bool
     :param gcp_conn_id: The connection ID to use when fetching connection info.
     :type gcp_conn_id: str
-    :param delegate_to: The account to impersonate, if any.
-        For this to work, the service account making the request must have
+    :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.
     :type delegate_to: str
     :param aws_conn_id: The connection id specifying the authentication information for the S3 Bucket.
     :type aws_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type google_impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = (
         'google_api_endpoint_params',
         's3_destination_key',
+        'google_impersonation_chain',
     )
     template_ext = ()
     ui_color = '#cc181e'
@@ -100,6 +111,7 @@ class GoogleApiToS3Operator(BaseOperator):
         gcp_conn_id='google_cloud_default',
         delegate_to=None,
         aws_conn_id='aws_default',
+        google_impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ):
         super().__init__(**kwargs)
@@ -117,6 +129,7 @@ class GoogleApiToS3Operator(BaseOperator):
         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
 
     def execute(self, context):
         """
@@ -142,7 +155,8 @@ class GoogleApiToS3Operator(BaseOperator):
             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
+            api_version=self.google_api_service_version,
+            impersonation_chain=self.google_impersonation_chain,
         )
         google_api_response = google_discovery_api_hook.query(
             endpoint=self.google_api_endpoint_path,
diff --git a/airflow/providers/google/ads/operators/ads.py b/airflow/providers/google/ads/operators/ads.py
index e47d509..c2071b7 100644
--- a/airflow/providers/google/ads/operators/ads.py
+++ b/airflow/providers/google/ads/operators/ads.py
@@ -20,7 +20,7 @@ This module contains Google Ad to GCS operators.
 """
 import csv
 from tempfile import NamedTemporaryFile
-from typing import Dict
+from typing import Dict, Optional, Sequence, Union
 
 from airflow.models import BaseOperator
 from airflow.providers.google.ads.hooks.ads import GoogleAdsHook
@@ -57,9 +57,18 @@ class GoogleAdsListAccountsOperator(BaseOperator):
     :type page_size: int
     :param gzip: Option to compress local file or file data for upload
     :type gzip: bool
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ("bucket", "object_name")
+    template_fields = ("bucket", "object_name", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -69,6 +78,7 @@ class GoogleAdsListAccountsOperator(BaseOperator):
         gcp_conn_id: str = "google_cloud_default",
         google_ads_conn_id: str = "google_ads_default",
         gzip: bool = False,
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs,
     ) -> None:
         super().__init__(**kwargs)
@@ -77,6 +87,7 @@ class GoogleAdsListAccountsOperator(BaseOperator):
         self.gcp_conn_id = gcp_conn_id
         self.google_ads_conn_id = google_ads_conn_id
         self.gzip = gzip
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context: Dict):
         uri = f"gs://{self.bucket}/{self.object_name}"
@@ -86,8 +97,10 @@ class GoogleAdsListAccountsOperator(BaseOperator):
             google_ads_conn_id=self.google_ads_conn_id
         )
 
-        gcs_hook = GCSHook(gcp_conn_id=self.gcp_conn_id)
-
+        gcs_hook = GCSHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain
+        )
         with NamedTemporaryFile("w+") as temp_file:
             # Download accounts
             accounts = ads_hook.list_accessible_customers()
diff --git a/airflow/providers/google/ads/transfers/ads_to_gcs.py b/airflow/providers/google/ads/transfers/ads_to_gcs.py
index 413c3b8..6b2fb71 100644
--- a/airflow/providers/google/ads/transfers/ads_to_gcs.py
+++ b/airflow/providers/google/ads/transfers/ads_to_gcs.py
@@ -18,7 +18,7 @@
 import csv
 from operator import attrgetter
 from tempfile import NamedTemporaryFile
-from typing import Dict, List
+from typing import Dict, List, Optional, Sequence, Union
 
 from airflow.models import BaseOperator
 from airflow.providers.google.ads.hooks.ads import GoogleAdsHook
@@ -58,9 +58,18 @@ class GoogleAdsToGcsOperator(BaseOperator):
     :type page_size: int
     :param gzip: Option to compress local file or file data for upload
     :type gzip: bool
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ("client_ids", "query", "attributes", "bucket", "obj")
+    template_fields = ("client_ids", "query", "attributes", "bucket", "obj", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -74,6 +83,7 @@ class GoogleAdsToGcsOperator(BaseOperator):
         google_ads_conn_id: str = "google_ads_default",
         page_size: int = 10000,
         gzip: bool = False,
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs,
     ) -> None:
         super().__init__(**kwargs)
@@ -86,6 +96,7 @@ class GoogleAdsToGcsOperator(BaseOperator):
         self.google_ads_conn_id = google_ads_conn_id
         self.page_size = page_size
         self.gzip = gzip
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context: Dict):
         service = GoogleAdsHook(
@@ -108,7 +119,10 @@ class GoogleAdsToGcsOperator(BaseOperator):
             writer.writerows(converted_rows)
             csvfile.flush()
 
-            hook = GCSHook(gcp_conn_id=self.gcp_conn_id)
+            hook = GCSHook(
+                gcp_conn_id=self.gcp_conn_id,
+                impersonation_chain=self.impersonation_chain
+            )
             hook.upload(
                 bucket_name=self.bucket,
                 object_name=self.obj,
diff --git a/airflow/providers/google/cloud/hooks/bigquery.py b/airflow/providers/google/cloud/hooks/bigquery.py
index e37e2ef..94cc123 100644
--- a/airflow/providers/google/cloud/hooks/bigquery.py
+++ b/airflow/providers/google/cloud/hooks/bigquery.py
@@ -63,11 +63,11 @@ class BigQueryHook(GoogleBaseHook, DbApiHook):
     def __init__(self,
                  gcp_conn_id: str = 'google_cloud_default',
                  delegate_to: Optional[str] = None,
-                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  use_legacy_sql: bool = True,
                  location: Optional[str] = None,
                  bigquery_conn_id: Optional[str] = None,
-                 api_resource_configs: Optional[Dict] = None) -> None:
+                 api_resource_configs: Optional[Dict] = None,
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,) -> None:
         # To preserve backward compatibility
         # TODO: remove one day
         if bigquery_conn_id:
diff --git a/airflow/providers/google/cloud/hooks/dataflow.py b/airflow/providers/google/cloud/hooks/dataflow.py
index 44dc877..569de11 100644
--- a/airflow/providers/google/cloud/hooks/dataflow.py
+++ b/airflow/providers/google/cloud/hooks/dataflow.py
@@ -416,8 +416,8 @@ class DataflowHook(GoogleBaseHook):
         self,
         gcp_conn_id: str = "google_cloud_default",
         delegate_to: Optional[str] = None,
+        poll_sleep: int = 10,
         impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
-        poll_sleep: int = 10
     ) -> None:
         self.poll_sleep = poll_sleep
         super().__init__(
diff --git a/airflow/providers/google/cloud/hooks/datastore.py b/airflow/providers/google/cloud/hooks/datastore.py
index 92de4bd..27685ec 100644
--- a/airflow/providers/google/cloud/hooks/datastore.py
+++ b/airflow/providers/google/cloud/hooks/datastore.py
@@ -44,9 +44,9 @@ class DatastoreHook(GoogleBaseHook):
         self,
         gcp_conn_id: str = "google_cloud_default",
         delegate_to: Optional[str] = None,
-        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         api_version: str = 'v1',
-        datastore_conn_id: Optional[str] = None
+        datastore_conn_id: Optional[str] = None,
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
     ) -> None:
         if datastore_conn_id:
             warnings.warn(
diff --git a/airflow/providers/google/cloud/hooks/gcs.py b/airflow/providers/google/cloud/hooks/gcs.py
index 949937b..470477f 100644
--- a/airflow/providers/google/cloud/hooks/gcs.py
+++ b/airflow/providers/google/cloud/hooks/gcs.py
@@ -116,8 +116,8 @@ class GCSHook(GoogleBaseHook):
         self,
         gcp_conn_id: str = "google_cloud_default",
         delegate_to: Optional[str] = None,
+        google_cloud_storage_conn_id: Optional[str] = None,
         impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
-        google_cloud_storage_conn_id: Optional[str] = None
     ) -> None:
         # To preserve backward compatibility
         # TODO: remove one day
diff --git a/airflow/providers/google/cloud/hooks/kubernetes_engine.py b/airflow/providers/google/cloud/hooks/kubernetes_engine.py
index a0e742e..099f48c 100644
--- a/airflow/providers/google/cloud/hooks/kubernetes_engine.py
+++ b/airflow/providers/google/cloud/hooks/kubernetes_engine.py
@@ -51,8 +51,8 @@ class GKEHook(GoogleBaseHook):
         self,
         gcp_conn_id: str = "google_cloud_default",
         delegate_to: Optional[str] = None,
+        location: Optional[str] = None,
         impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
-        location: Optional[str] = None
     ) -> None:
         super().__init__(
             gcp_conn_id=gcp_conn_id,
diff --git a/airflow/providers/google/cloud/operators/automl.py b/airflow/providers/google/cloud/operators/automl.py
index 9d88b16..38fd11d 100644
--- a/airflow/providers/google/cloud/operators/automl.py
+++ b/airflow/providers/google/cloud/operators/automl.py
@@ -58,9 +58,18 @@ class AutoMLTrainModelOperator(BaseOperator):
     :type metadata: Optional[Sequence[Tuple[str, str]]]
     :param gcp_conn_id: The connection ID to use to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ("model", "location", "project_id")
+    template_fields = ("model", "location", "project_id", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -72,6 +81,7 @@ class AutoMLTrainModelOperator(BaseOperator):
         timeout: Optional[float] = None,
         retry: Optional[Retry] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -83,9 +93,13 @@ class AutoMLTrainModelOperator(BaseOperator):
         self.timeout = timeout
         self.retry = retry
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudAutoMLHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudAutoMLHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         self.log.info("Creating model.")
         operation = hook.create_model(
             model=self.model,
@@ -132,9 +146,18 @@ class AutoMLPredictOperator(BaseOperator):
     :type metadata: Optional[Sequence[Tuple[str, str]]]
     :param gcp_conn_id: The connection ID to use to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ("model_id", "location", "project_id")
+    template_fields = ("model_id", "location", "project_id", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -148,6 +171,7 @@ class AutoMLPredictOperator(BaseOperator):
         timeout: Optional[float] = None,
         retry: Optional[Retry] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -161,9 +185,13 @@ class AutoMLPredictOperator(BaseOperator):
         self.retry = retry
         self.payload = payload
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudAutoMLHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudAutoMLHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         result = hook.predict(
             model_id=self.model_id,
             payload=self.payload,
@@ -218,6 +246,15 @@ class AutoMLBatchPredictOperator(BaseOperator):
     :type metadata: Optional[Sequence[Tuple[str, str]]]
     :param gcp_conn_id: The connection ID to use to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = (
@@ -226,6 +263,7 @@ class AutoMLBatchPredictOperator(BaseOperator):
         "output_config",
         "location",
         "project_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -241,6 +279,7 @@ class AutoMLBatchPredictOperator(BaseOperator):
         timeout: Optional[float] = None,
         retry: Optional[Retry] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -253,11 +292,15 @@ class AutoMLBatchPredictOperator(BaseOperator):
         self.timeout = timeout
         self.retry = retry
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
         self.input_config = input_config
         self.output_config = output_config
 
     def execute(self, context):
-        hook = CloudAutoMLHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudAutoMLHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         self.log.info("Fetch batch prediction.")
         operation = hook.batch_predict(
             model_id=self.model_id,
@@ -303,9 +346,18 @@ class AutoMLCreateDatasetOperator(BaseOperator):
     :type metadata: Optional[Sequence[Tuple[str, str]]]
     :param gcp_conn_id: The connection ID to use to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ("dataset", "location", "project_id")
+    template_fields = ("dataset", "location", "project_id", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -317,6 +369,7 @@ class AutoMLCreateDatasetOperator(BaseOperator):
         timeout: Optional[float] = None,
         retry: Optional[Retry] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -328,9 +381,13 @@ class AutoMLCreateDatasetOperator(BaseOperator):
         self.timeout = timeout
         self.retry = retry
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudAutoMLHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudAutoMLHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         self.log.info("Creating dataset")
         result = hook.create_dataset(
             dataset=self.dataset,
@@ -378,9 +435,19 @@ class AutoMLImportDataOperator(BaseOperator):
     :type metadata: Optional[Sequence[Tuple[str, str]]]
     :param gcp_conn_id: The connection ID to use to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ("dataset_id", "input_config", "location", "project_id")
+    template_fields = ("dataset_id", "input_config", "location", "project_id",
+                       "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -393,6 +460,7 @@ class AutoMLImportDataOperator(BaseOperator):
         timeout: Optional[float] = None,
         retry: Optional[Retry] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -405,9 +473,13 @@ class AutoMLImportDataOperator(BaseOperator):
         self.timeout = timeout
         self.retry = retry
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudAutoMLHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudAutoMLHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         self.log.info("Importing dataset")
         operation = hook.import_data(
             dataset_id=self.dataset_id,
@@ -461,6 +533,15 @@ class AutoMLTablesListColumnSpecsOperator(BaseOperator):
     :type metadata: Optional[Sequence[Tuple[str, str]]]
     :param gcp_conn_id: The connection ID to use to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = (
@@ -470,6 +551,7 @@ class AutoMLTablesListColumnSpecsOperator(BaseOperator):
         "filter_",
         "location",
         "project_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -486,6 +568,7 @@ class AutoMLTablesListColumnSpecsOperator(BaseOperator):
         timeout: Optional[float] = None,
         retry: Optional[Retry] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -500,9 +583,13 @@ class AutoMLTablesListColumnSpecsOperator(BaseOperator):
         self.timeout = timeout
         self.retry = retry
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudAutoMLHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudAutoMLHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         self.log.info("Requesting column specs.")
         page_iterator = hook.list_column_specs(
             dataset_id=self.dataset_id,
@@ -550,9 +637,18 @@ class AutoMLTablesUpdateDatasetOperator(BaseOperator):
     :type metadata: Optional[Sequence[Tuple[str, str]]]
     :param gcp_conn_id: The connection ID to use to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ("dataset", "update_mask", "location")
+    template_fields = ("dataset", "update_mask", "location", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -564,6 +660,7 @@ class AutoMLTablesUpdateDatasetOperator(BaseOperator):
         timeout: Optional[float] = None,
         retry: Optional[Retry] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -575,9 +672,13 @@ class AutoMLTablesUpdateDatasetOperator(BaseOperator):
         self.timeout = timeout
         self.retry = retry
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudAutoMLHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudAutoMLHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         self.log.info("Updating AutoML dataset %s.", self.dataset["name"])
         result = hook.update_dataset(
             dataset=self.dataset,
@@ -617,9 +718,18 @@ class AutoMLGetModelOperator(BaseOperator):
     :type metadata: Optional[Sequence[Tuple[str, str]]]
     :param gcp_conn_id: The connection ID to use to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ("model_id", "location", "project_id")
+    template_fields = ("model_id", "location", "project_id", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -631,6 +741,7 @@ class AutoMLGetModelOperator(BaseOperator):
         timeout: Optional[float] = None,
         retry: Optional[Retry] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -642,9 +753,13 @@ class AutoMLGetModelOperator(BaseOperator):
         self.timeout = timeout
         self.retry = retry
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudAutoMLHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudAutoMLHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         result = hook.get_model(
             model_id=self.model_id,
             location=self.location,
@@ -683,9 +798,18 @@ class AutoMLDeleteModelOperator(BaseOperator):
     :type metadata: Optional[Sequence[Tuple[str, str]]]
     :param gcp_conn_id: The connection ID to use to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ("model_id", "location", "project_id")
+    template_fields = ("model_id", "location", "project_id", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -697,6 +821,7 @@ class AutoMLDeleteModelOperator(BaseOperator):
         timeout: Optional[float] = None,
         retry: Optional[Retry] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -708,9 +833,13 @@ class AutoMLDeleteModelOperator(BaseOperator):
         self.timeout = timeout
         self.retry = retry
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudAutoMLHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudAutoMLHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         operation = hook.delete_model(
             model_id=self.model_id,
             location=self.location,
@@ -759,9 +888,18 @@ class AutoMLDeployModelOperator(BaseOperator):
     :type metadata: Optional[Sequence[Tuple[str, str]]]
     :param gcp_conn_id: The connection ID to use to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ("model_id", "location", "project_id")
+    template_fields = ("model_id", "location", "project_id", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -774,6 +912,7 @@ class AutoMLDeployModelOperator(BaseOperator):
         timeout: Optional[float] = None,
         retry: Optional[Retry] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -786,9 +925,13 @@ class AutoMLDeployModelOperator(BaseOperator):
         self.timeout = timeout
         self.retry = retry
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudAutoMLHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudAutoMLHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         self.log.info("Deploying model_id %s", self.model_id)
 
         operation = hook.deploy_model(
@@ -838,9 +981,18 @@ class AutoMLTablesListTableSpecsOperator(BaseOperator):
     :type metadata: Optional[Sequence[Tuple[str, str]]]
     :param gcp_conn_id: The connection ID to use to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ("dataset_id", "filter_", "location", "project_id")
+    template_fields = ("dataset_id", "filter_", "location", "project_id", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -854,6 +1006,7 @@ class AutoMLTablesListTableSpecsOperator(BaseOperator):
         timeout: Optional[float] = None,
         retry: Optional[Retry] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -866,9 +1019,13 @@ class AutoMLTablesListTableSpecsOperator(BaseOperator):
         self.timeout = timeout
         self.retry = retry
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudAutoMLHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudAutoMLHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         self.log.info("Requesting table specs for %s.", self.dataset_id)
         page_iterator = hook.list_table_specs(
             dataset_id=self.dataset_id,
@@ -909,9 +1066,18 @@ class AutoMLListDatasetOperator(BaseOperator):
     :type metadata: Optional[Sequence[Tuple[str, str]]]
     :param gcp_conn_id: The connection ID to use to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ("location", "project_id")
+    template_fields = ("location", "project_id", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -922,6 +1088,7 @@ class AutoMLListDatasetOperator(BaseOperator):
         timeout: Optional[float] = None,
         retry: Optional[Retry] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -931,9 +1098,13 @@ class AutoMLListDatasetOperator(BaseOperator):
         self.timeout = timeout
         self.retry = retry
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudAutoMLHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudAutoMLHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         self.log.info("Requesting datasets")
         page_iterator = hook.list_datasets(
             location=self.location,
@@ -979,9 +1150,18 @@ class AutoMLDeleteDatasetOperator(BaseOperator):
     :type metadata: Optional[Sequence[Tuple[str, str]]]
     :param gcp_conn_id: The connection ID to use to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ("dataset_id", "location", "project_id")
+    template_fields = ("dataset_id", "location", "project_id", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -993,6 +1173,7 @@ class AutoMLDeleteDatasetOperator(BaseOperator):
         timeout: Optional[float] = None,
         retry: Optional[Retry] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -1004,6 +1185,7 @@ class AutoMLDeleteDatasetOperator(BaseOperator):
         self.timeout = timeout
         self.retry = retry
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     @staticmethod
     def _parse_dataset_id(dataset_id: Union[str, List[str]]) -> List[str]:
@@ -1015,7 +1197,10 @@ class AutoMLDeleteDatasetOperator(BaseOperator):
             return dataset_id.split(",")
 
     def execute(self, context):
-        hook = CloudAutoMLHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudAutoMLHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         dataset_id_list = self._parse_dataset_id(self.dataset_id)
         for dataset_id in dataset_id_list:
             self.log.info("Deleting dataset %s", dataset_id)
diff --git a/airflow/providers/google/cloud/operators/bigquery.py b/airflow/providers/google/cloud/operators/bigquery.py
index e08a09d..e02601b 100644
--- a/airflow/providers/google/cloud/operators/bigquery.py
+++ b/airflow/providers/google/cloud/operators/bigquery.py
@@ -26,7 +26,7 @@ import json
 import re
 import uuid
 import warnings
-from typing import Any, Dict, Iterable, List, Optional, Set, SupportsAbs, Union
+from typing import Any, Dict, Iterable, List, Optional, Sequence, Set, SupportsAbs, Union
 
 import attr
 from google.api_core.exceptions import Conflict
@@ -134,9 +134,18 @@ class BigQueryCheckOperator(CheckOperator):
     :param location: The geographic location of the job. See details at:
         https://cloud.google.com/bigquery/docs/locations#specifying_your_location
     :type location: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ('sql', 'gcp_conn_id',)
+    template_fields = ('sql', 'gcp_conn_id', 'impersonation_chain',)
     template_ext = ('.sql',)
     ui_color = BigQueryUIColors.CHECK.value
 
@@ -148,6 +157,7 @@ class BigQueryCheckOperator(CheckOperator):
         bigquery_conn_id: Optional[str] = None,
         use_legacy_sql: bool = True,
         location: Optional[str] = None,
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs,
     ) -> None:
         super().__init__(sql=sql, **kwargs)
@@ -159,12 +169,14 @@ class BigQueryCheckOperator(CheckOperator):
         self.sql = sql
         self.use_legacy_sql = use_legacy_sql
         self.location = location
+        self.impersonation_chain = impersonation_chain
 
     def get_db_hook(self) -> BigQueryHook:
         return BigQueryHook(
             gcp_conn_id=self.gcp_conn_id,
             use_legacy_sql=self.use_legacy_sql,
-            location=self.location
+            location=self.location,
+            impersonation_chain=self.impersonation_chain,
         )
 
 
@@ -189,9 +201,18 @@ class BigQueryValueCheckOperator(ValueCheckOperator):
     :param location: The geographic location of the job. See details at:
         https://cloud.google.com/bigquery/docs/locations#specifying_your_location
     :type location: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ('sql', 'gcp_conn_id', 'pass_value',)
+    template_fields = ('sql', 'gcp_conn_id', 'pass_value', 'impersonation_chain',)
     template_ext = ('.sql',)
     ui_color = BigQueryUIColors.CHECK.value
 
@@ -205,6 +226,7 @@ class BigQueryValueCheckOperator(ValueCheckOperator):
         bigquery_conn_id: Optional[str] = None,
         use_legacy_sql: bool = True,
         location: Optional[str] = None,
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs,
     ) -> None:
         super().__init__(
@@ -221,12 +243,14 @@ class BigQueryValueCheckOperator(ValueCheckOperator):
         self.location = location
         self.gcp_conn_id = gcp_conn_id
         self.use_legacy_sql = use_legacy_sql
+        self.impersonation_chain = impersonation_chain
 
     def get_db_hook(self) -> BigQueryHook:
         return BigQueryHook(
             gcp_conn_id=self.gcp_conn_id,
             use_legacy_sql=self.use_legacy_sql,
-            location=self.location
+            location=self.location,
+            impersonation_chain=self.impersonation_chain,
         )
 
 
@@ -264,9 +288,18 @@ class BigQueryIntervalCheckOperator(IntervalCheckOperator):
     :param location: The geographic location of the job. See details at:
         https://cloud.google.com/bigquery/docs/locations#specifying_your_location
     :type location: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ('table', 'gcp_conn_id', 'sql1', 'sql2')
+    template_fields = ('table', 'gcp_conn_id', 'sql1', 'sql2', 'impersonation_chain',)
     ui_color = BigQueryUIColors.CHECK.value
 
     @apply_defaults
@@ -280,6 +313,7 @@ class BigQueryIntervalCheckOperator(IntervalCheckOperator):
         bigquery_conn_id: Optional[str] = None,
         use_legacy_sql: bool = True,
         location: Optional[str] = None,
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs,
     ) -> None:
         super().__init__(
@@ -297,12 +331,14 @@ class BigQueryIntervalCheckOperator(IntervalCheckOperator):
         self.gcp_conn_id = gcp_conn_id
         self.use_legacy_sql = use_legacy_sql
         self.location = location
+        self.impersonation_chain = impersonation_chain
 
     def get_db_hook(self) -> BigQueryHook:
         return BigQueryHook(
             gcp_conn_id=self.gcp_conn_id,
             use_legacy_sql=self.use_legacy_sql,
             location=self.location,
+            impersonation_chain=self.impersonation_chain,
         )
 
 
@@ -353,14 +389,24 @@ class BigQueryGetDataOperator(BaseOperator):
     :param bigquery_conn_id: (Deprecated) The connection ID used to connect to Google Cloud Platform.
         This parameter has been deprecated. You should pass the gcp_conn_id parameter instead.
     :type bigquery_conn_id: str
-    :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 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.
     :type delegate_to: str
     :param location: The location used for the operation.
     :type location: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
-    template_fields = ('dataset_id', 'table_id', 'max_results', 'selected_fields')
+    template_fields = ('dataset_id', 'table_id', 'max_results', 'selected_fields',
+                       'impersonation_chain',)
     ui_color = BigQueryUIColors.QUERY.value
 
     @apply_defaults
@@ -374,6 +420,7 @@ class BigQueryGetDataOperator(BaseOperator):
         bigquery_conn_id: Optional[str] = None,
         delegate_to: Optional[str] = None,
         location: Optional[str] = None,
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -391,6 +438,7 @@ class BigQueryGetDataOperator(BaseOperator):
         self.gcp_conn_id = gcp_conn_id
         self.delegate_to = delegate_to
         self.location = location
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
         self.log.info('Fetching Data from %s.%s max results: %s',
@@ -398,7 +446,8 @@ class BigQueryGetDataOperator(BaseOperator):
 
         hook = BigQueryHook(
             bigquery_conn_id=self.gcp_conn_id,
-            delegate_to=self.delegate_to
+            delegate_to=self.delegate_to,
+            impersonation_chain=self.impersonation_chain,
         )
 
         rows = hook.list_rows(
@@ -447,9 +496,9 @@ class BigQueryExecuteQueryOperator(BaseOperator):
     :param bigquery_conn_id: (Deprecated) The connection ID used to connect to Google Cloud Platform.
         This parameter has been deprecated. You should pass the gcp_conn_id parameter instead.
     :type bigquery_conn_id: str
-    :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 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.
     :type delegate_to: str
     :param udf_config: The User Defined Function configuration for the query.
         See https://cloud.google.com/bigquery/user-defined-functions for details.
@@ -507,9 +556,19 @@ class BigQueryExecuteQueryOperator(BaseOperator):
                 "kmsKeyName": "projects/testp/locations/us/keyRings/test-kr/cryptoKeys/test-key"
             }
     :type encryption_configuration: dict
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ('sql', 'destination_dataset_table', 'labels', 'query_params')
+    template_fields = ('sql', 'destination_dataset_table', 'labels', 'query_params',
+                       'impersonation_chain',)
     template_ext = ('.sql', )
     ui_color = BigQueryUIColors.QUERY.value
 
@@ -552,6 +611,7 @@ class BigQueryExecuteQueryOperator(BaseOperator):
                  cluster_fields: Optional[List[str]] = None,
                  location: Optional[str] = None,
                  encryption_configuration: Optional[dict] = None,
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         super().__init__(**kwargs)
 
@@ -588,6 +648,7 @@ class BigQueryExecuteQueryOperator(BaseOperator):
         self.location = location
         self.encryption_configuration = encryption_configuration
         self.hook = None  # type: Optional[BigQueryHook]
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
         if self.hook is None:
@@ -597,6 +658,7 @@ class BigQueryExecuteQueryOperator(BaseOperator):
                 use_legacy_sql=self.use_legacy_sql,
                 delegate_to=self.delegate_to,
                 location=self.location,
+                impersonation_chain=self.impersonation_chain,
             )
         if isinstance(self.sql, str):
             job_id = self.hook.run_query(
@@ -702,9 +764,9 @@ class BigQueryCreateEmptyTableOperator(BaseOperator):
     :param google_cloud_storage_conn_id: [Optional] The connection ID used to connect to Google Cloud
         Platform and interact with the Google Cloud Storage service.
     :type google_cloud_storage_conn_id: str
-    :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 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.
     :type delegate_to: str
     :param labels: a dictionary containing labels for the table, passed to BigQuery
 
@@ -770,6 +832,15 @@ class BigQueryCreateEmptyTableOperator(BaseOperator):
             .. seealso::
                 https://cloud.google.com/bigquery/docs/reference/rest/v2/tables#clustering.fields
     :type cluster_fields: list
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
     template_fields = (
         'dataset_id',
@@ -777,7 +848,8 @@ class BigQueryCreateEmptyTableOperator(BaseOperator):
         'project_id',
         'gcs_schema_object',
         'labels',
-        'view'
+        'view',
+        'impersonation_chain',
     )
 
     ui_color = BigQueryUIColors.TABLE.value
@@ -801,6 +873,7 @@ class BigQueryCreateEmptyTableOperator(BaseOperator):
         encryption_configuration: Optional[Dict] = None,
         location: Optional[str] = None,
         cluster_fields: Optional[List[str]] = None,
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -820,19 +893,23 @@ class BigQueryCreateEmptyTableOperator(BaseOperator):
         self.location = location
         self.cluster_fields = cluster_fields
         self.table_resource = table_resource
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
         bq_hook = BigQueryHook(
             gcp_conn_id=self.bigquery_conn_id,
             delegate_to=self.delegate_to,
-            location=self.location
+            location=self.location,
+            impersonation_chain=self.impersonation_chain,
         )
 
         if not self.schema_fields and self.gcs_schema_object:
             gcs_bucket, gcs_object = _parse_gcs_url(self.gcs_schema_object)
             gcs_hook = GCSHook(
                 google_cloud_storage_conn_id=self.google_cloud_storage_conn_id,
-                delegate_to=self.delegate_to)
+                delegate_to=self.delegate_to,
+                impersonation_chain=self.impersonation_chain,
+            )
             schema_fields = json.loads(gcs_hook.download(
                 gcs_bucket,
                 gcs_object).decode("utf-8"))
@@ -933,9 +1010,9 @@ class BigQueryCreateExternalTableOperator(BaseOperator):
         Platform and interact with the Google Cloud Storage service.
         cloud storage hook.
     :type google_cloud_storage_conn_id: str
-    :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 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.
     :type delegate_to: str
     :param src_fmt_configs: configure optional fields specific to the source format
     :type src_fmt_configs: dict
@@ -950,6 +1027,15 @@ class BigQueryCreateExternalTableOperator(BaseOperator):
     :type encryption_configuration: dict
     :param location: The location used for the operation.
     :type location: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
     template_fields = (
         'bucket',
@@ -958,6 +1044,7 @@ class BigQueryCreateExternalTableOperator(BaseOperator):
         'destination_project_dataset_table',
         'labels',
         'table_resource',
+        'impersonation_chain',
     )
     ui_color = BigQueryUIColors.TABLE.value
 
@@ -986,6 +1073,7 @@ class BigQueryCreateExternalTableOperator(BaseOperator):
         labels: Optional[Dict] = None,
         encryption_configuration: Optional[Dict] = None,
         location: Optional[str] = None,
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -1043,18 +1131,22 @@ class BigQueryCreateExternalTableOperator(BaseOperator):
         self.labels = labels
         self.encryption_configuration = encryption_configuration
         self.location = location
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
         bq_hook = BigQueryHook(
             gcp_conn_id=self.bigquery_conn_id,
             delegate_to=self.delegate_to,
-            location=self.location
+            location=self.location,
+            impersonation_chain=self.impersonation_chain,
         )
 
         if not self.schema_fields and self.schema_object and self.source_format != 'DATASTORE_BACKUP':
             gcs_hook = GCSHook(
                 google_cloud_storage_conn_id=self.google_cloud_storage_conn_id,
-                delegate_to=self.delegate_to)
+                delegate_to=self.delegate_to,
+                impersonation_chain=self.impersonation_chain,
+            )
             schema_object = gcs_hook.download(self.bucket, self.schema_object)
             schema_fields = json.loads(schema_object.decode("utf-8"))
         else:
@@ -1115,6 +1207,19 @@ class BigQueryDeleteDatasetOperator(BaseOperator):
     :param bigquery_conn_id: (Deprecated) The connection ID used to connect to Google Cloud Platform.
         This parameter has been deprecated. You should pass the gcp_conn_id parameter instead.
     :type bigquery_conn_id: str
+    :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.
+    :type delegate_to: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
 
     **Example**: ::
 
@@ -1127,7 +1232,7 @@ class BigQueryDeleteDatasetOperator(BaseOperator):
             dag=dag)
     """
 
-    template_fields = ('dataset_id', 'project_id')
+    template_fields = ('dataset_id', 'project_id', 'impersonation_chain',)
     ui_color = BigQueryUIColors.DATASET.value
 
     @apply_defaults
@@ -1139,6 +1244,7 @@ class BigQueryDeleteDatasetOperator(BaseOperator):
         gcp_conn_id: str = 'google_cloud_default',
         bigquery_conn_id: Optional[str] = None,
         delegate_to: Optional[str] = None,
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         if bigquery_conn_id:
@@ -1152,6 +1258,7 @@ class BigQueryDeleteDatasetOperator(BaseOperator):
         self.delete_contents = delete_contents
         self.gcp_conn_id = gcp_conn_id
         self.delegate_to = delegate_to
+        self.impersonation_chain = impersonation_chain
 
         super().__init__(**kwargs)
 
@@ -1160,7 +1267,8 @@ class BigQueryDeleteDatasetOperator(BaseOperator):
 
         bq_hook = BigQueryHook(
             gcp_conn_id=self.gcp_conn_id,
-            delegate_to=self.delegate_to
+            delegate_to=self.delegate_to,
+            impersonation_chain=self.impersonation_chain,
         )
 
         bq_hook.delete_dataset(
@@ -1194,6 +1302,19 @@ class BigQueryCreateEmptyDatasetOperator(BaseOperator):
     :param bigquery_conn_id: (Deprecated) The connection ID used to connect to Google Cloud Platform.
         This parameter has been deprecated. You should pass the gcp_conn_id parameter instead.
     :type bigquery_conn_id: str
+    :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.
+    :type delegate_to: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
         **Example**: ::
 
             create_new_dataset = BigQueryCreateEmptyDatasetOperator(
@@ -1203,11 +1324,9 @@ class BigQueryCreateEmptyDatasetOperator(BaseOperator):
                 gcp_conn_id='_my_gcp_conn_',
                 task_id='newDatasetCreator',
                 dag=dag)
-    :param location: The location used for the operation.
-    :type location: str
     """
 
-    template_fields = ('dataset_id', 'project_id', 'dataset_reference')
+    template_fields = ('dataset_id', 'project_id', 'dataset_reference', 'impersonation_chain',)
     ui_color = BigQueryUIColors.DATASET.value
 
     @apply_defaults
@@ -1220,6 +1339,7 @@ class BigQueryCreateEmptyDatasetOperator(BaseOperator):
                  gcp_conn_id: str = 'google_cloud_default',
                  bigquery_conn_id: Optional[str] = None,
                  delegate_to: Optional[str] = None,
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
 
         if bigquery_conn_id:
@@ -1234,6 +1354,7 @@ class BigQueryCreateEmptyDatasetOperator(BaseOperator):
         self.gcp_conn_id = gcp_conn_id
         self.dataset_reference = dataset_reference if dataset_reference else {}
         self.delegate_to = delegate_to
+        self.impersonation_chain = impersonation_chain
 
         super().__init__(**kwargs)
 
@@ -1241,7 +1362,8 @@ class BigQueryCreateEmptyDatasetOperator(BaseOperator):
         bq_hook = BigQueryHook(
             gcp_conn_id=self.gcp_conn_id,
             delegate_to=self.delegate_to,
-            location=self.location
+            location=self.location,
+            impersonation_chain=self.impersonation_chain,
         )
 
         try:
@@ -1273,11 +1395,25 @@ class BigQueryGetDatasetOperator(BaseOperator):
     :type project_id: str
     :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+    :type delegate_to: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
+
     :rtype: dataset
         https://cloud.google.com/bigquery/docs/reference/rest/v2/datasets#resource
     """
 
-    template_fields = ('dataset_id', 'project_id')
+    template_fields = ('dataset_id', 'project_id', 'impersonation_chain',)
     ui_color = BigQueryUIColors.DATASET.value
 
     @apply_defaults
@@ -1287,16 +1423,20 @@ class BigQueryGetDatasetOperator(BaseOperator):
                  project_id: Optional[str] = None,
                  gcp_conn_id: str = 'google_cloud_default',
                  delegate_to: Optional[str] = None,
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
+
         self.dataset_id = dataset_id
         self.project_id = project_id
         self.gcp_conn_id = gcp_conn_id
         self.delegate_to = delegate_to
+        self.impersonation_chain = impersonation_chain
         super().__init__(**kwargs)
 
     def execute(self, context):
         bq_hook = BigQueryHook(gcp_conn_id=self.gcp_conn_id,
-                               delegate_to=self.delegate_to)
+                               delegate_to=self.delegate_to,
+                               impersonation_chain=self.impersonation_chain)
 
         self.log.info('Start getting dataset: %s:%s', self.project_id, self.dataset_id)
 
@@ -1323,12 +1463,21 @@ class BigQueryGetDatasetTablesOperator(BaseOperator):
     :type max_results: int
     :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
-    :param delegate_to: (Optional) The account to impersonate, if any.
-        For this to work, the service account making the request must have domain-wide
-        delegation 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.
     :type delegate_to: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
-    template_fields = ('dataset_id', 'project_id')
+    template_fields = ('dataset_id', 'project_id', 'impersonation_chain',)
     ui_color = BigQueryUIColors.DATASET.value
 
     @apply_defaults
@@ -1339,6 +1488,7 @@ class BigQueryGetDatasetTablesOperator(BaseOperator):
         max_results: Optional[int] = None,
         gcp_conn_id: Optional[str] = 'google_cloud_default',
         delegate_to: Optional[str] = None,
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         self.dataset_id = dataset_id
@@ -1346,12 +1496,14 @@ class BigQueryGetDatasetTablesOperator(BaseOperator):
         self.max_results = max_results
         self.gcp_conn_id = gcp_conn_id
         self.delegate_to = delegate_to
+        self.impersonation_chain = impersonation_chain
         super().__init__(**kwargs)
 
     def execute(self, context):
         bq_hook = BigQueryHook(
             gcp_conn_id=self.gcp_conn_id,
             delegate_to=self.delegate_to,
+            impersonation_chain=self.impersonation_chain,
         )
 
         return bq_hook.get_dataset_tables(
@@ -1381,11 +1533,25 @@ class BigQueryPatchDatasetOperator(BaseOperator):
     :type project_id: str
     :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+    :type delegate_to: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
+
     :rtype: dataset
         https://cloud.google.com/bigquery/docs/reference/rest/v2/datasets#resource
     """
 
-    template_fields = ('dataset_id', 'project_id')
+    template_fields = ('dataset_id', 'project_id', 'impersonation_chain',)
     ui_color = BigQueryUIColors.DATASET.value
 
     @apply_defaults
@@ -1396,6 +1562,7 @@ class BigQueryPatchDatasetOperator(BaseOperator):
         project_id: Optional[str] = None,
         gcp_conn_id: str = 'google_cloud_default',
         delegate_to: Optional[str] = None,
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs,
     ) -> None:
 
@@ -1408,12 +1575,14 @@ class BigQueryPatchDatasetOperator(BaseOperator):
         self.gcp_conn_id = gcp_conn_id
         self.dataset_resource = dataset_resource
         self.delegate_to = delegate_to
+        self.impersonation_chain = impersonation_chain
         super().__init__(**kwargs)
 
     def execute(self, context):
         bq_hook = BigQueryHook(
             gcp_conn_id=self.gcp_conn_id,
             delegate_to=self.delegate_to,
+            impersonation_chain=self.impersonation_chain,
         )
 
         return bq_hook.patch_dataset(
@@ -1448,11 +1617,25 @@ class BigQueryUpdateDatasetOperator(BaseOperator):
     :type project_id: str
     :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+    :type delegate_to: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
+
     :rtype: dataset
         https://cloud.google.com/bigquery/docs/reference/rest/v2/datasets#resource
     """
 
-    template_fields = ('dataset_id', 'project_id')
+    template_fields = ('dataset_id', 'project_id', 'impersonation_chain',)
     ui_color = BigQueryUIColors.DATASET.value
 
     @apply_defaults
@@ -1464,6 +1647,7 @@ class BigQueryUpdateDatasetOperator(BaseOperator):
         project_id: Optional[str] = None,
         gcp_conn_id: str = 'google_cloud_default',
         delegate_to: Optional[str] = None,
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         self.dataset_id = dataset_id
@@ -1472,12 +1656,14 @@ class BigQueryUpdateDatasetOperator(BaseOperator):
         self.gcp_conn_id = gcp_conn_id
         self.dataset_resource = dataset_resource
         self.delegate_to = delegate_to
+        self.impersonation_chain = impersonation_chain
         super().__init__(**kwargs)
 
     def execute(self, 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())
 
@@ -1507,17 +1693,26 @@ class BigQueryDeleteTableOperator(BaseOperator):
     :param bigquery_conn_id: (Deprecated) The connection ID used to connect to Google Cloud Platform.
         This parameter has been deprecated. You should pass the gcp_conn_id parameter instead.
     :type bigquery_conn_id: str
-    :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 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.
     :type delegate_to: str
     :param ignore_if_missing: if True, then return success even if the
         requested table does not exist.
     :type ignore_if_missing: bool
     :param location: The location used for the operation.
     :type location: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
-    template_fields = ('deletion_dataset_table',)
+    template_fields = ('deletion_dataset_table', 'impersonation_chain',)
     ui_color = BigQueryUIColors.TABLE.value
 
     @apply_defaults
@@ -1529,6 +1724,7 @@ class BigQueryDeleteTableOperator(BaseOperator):
         delegate_to: Optional[str] = None,
         ignore_if_missing: bool = False,
         location: Optional[str] = None,
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs,
     ) -> None:
         super().__init__(**kwargs)
@@ -1544,13 +1740,15 @@ class BigQueryDeleteTableOperator(BaseOperator):
         self.delegate_to = delegate_to
         self.ignore_if_missing = ignore_if_missing
         self.location = location
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
         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
+            location=self.location,
+            impersonation_chain=self.impersonation_chain,
         )
         hook.delete_table(
             table_id=self.deletion_dataset_table,
@@ -1587,8 +1785,17 @@ class BigQueryUpsertTableOperator(BaseOperator):
     :type delegate_to: str
     :param location: The location used for the operation.
     :type location: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
-    template_fields = ('dataset_id', 'table_resource')
+    template_fields = ('dataset_id', 'table_resource', 'impersonation_chain',)
     ui_color = BigQueryUIColors.TABLE.value
 
     @apply_defaults
@@ -1601,6 +1808,7 @@ class BigQueryUpsertTableOperator(BaseOperator):
         bigquery_conn_id: Optional[str] = None,
         delegate_to: Optional[str] = None,
         location: Optional[str] = None,
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs,
     ) -> None:
         super().__init__(**kwargs)
@@ -1617,6 +1825,7 @@ class BigQueryUpsertTableOperator(BaseOperator):
         self.gcp_conn_id = gcp_conn_id
         self.delegate_to = delegate_to
         self.location = location
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
         self.log.info('Upserting Dataset: %s with table_resource: %s', self.dataset_id, self.table_resource)
@@ -1624,6 +1833,7 @@ class BigQueryUpsertTableOperator(BaseOperator):
             bigquery_conn_id=self.gcp_conn_id,
             delegate_to=self.delegate_to,
             location=self.location,
+            impersonation_chain=self.impersonation_chain,
         )
         hook.run_table_upsert(
             dataset_id=self.dataset_id,
@@ -1675,9 +1885,22 @@ class BigQueryInsertJobOperator(BaseOperator):
     :type location: str
     :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+    :type delegate_to: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ("configuration", "job_id")
+    template_fields = ("configuration", "job_id", "impersonation_chain",)
     template_ext = (".json", )
     ui_color = BigQueryUIColors.QUERY.value
 
@@ -1691,6 +1914,7 @@ class BigQueryInsertJobOperator(BaseOperator):
         reattach_states: Optional[Set[str]] = None,
         gcp_conn_id: str = 'google_cloud_default',
         delegate_to: Optional[str] = None,
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs,
     ) -> None:
         super().__init__(**kwargs)
@@ -1702,6 +1926,7 @@ class BigQueryInsertJobOperator(BaseOperator):
         self.delegate_to = delegate_to
         self.force_rerun = force_rerun
         self.reattach_states: Set[str] = reattach_states or set()
+        self.impersonation_chain = impersonation_chain
 
     def prepare_template(self) -> None:
         # If .json is passed then we have to read the file
@@ -1748,6 +1973,7 @@ class BigQueryInsertJobOperator(BaseOperator):
         hook = BigQueryHook(
             gcp_conn_id=self.gcp_conn_id,
             delegate_to=self.delegate_to,
+            impersonation_chain=self.impersonation_chain,
         )
 
         job_id = self._job_id(context)
diff --git a/airflow/providers/google/cloud/operators/bigquery_dts.py b/airflow/providers/google/cloud/operators/bigquery_dts.py
index 3acfc48..50e96c7 100644
--- a/airflow/providers/google/cloud/operators/bigquery_dts.py
+++ b/airflow/providers/google/cloud/operators/bigquery_dts.py
@@ -18,7 +18,7 @@
 """
 This module contains Google BigQuery Data Transfer Service operators.
 """
-from typing import Optional, Sequence, Tuple
+from typing import Optional, Sequence, Tuple, Union
 
 from google.api_core.retry import Retry
 from google.protobuf.json_format import MessageToDict
@@ -55,6 +55,15 @@ class BigQueryCreateDataTransferOperator(BaseOperator):
     :type metadata: Optional[Sequence[Tuple[str, str]]]
     :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = (
@@ -62,6 +71,7 @@ class BigQueryCreateDataTransferOperator(BaseOperator):
         "project_id",
         "authorization_code",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -74,6 +84,7 @@ class BigQueryCreateDataTransferOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id="google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ):
         super().__init__(**kwargs)
@@ -84,9 +95,13 @@ class BigQueryCreateDataTransferOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = BiqQueryDataTransferServiceHook(gcp_conn_id=self.gcp_conn_id)
+        hook = BiqQueryDataTransferServiceHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain
+        )
         self.log.info("Creating DTS transfer config")
         response = hook.create_transfer_config(
             project_id=self.project_id,
@@ -126,9 +141,18 @@ class BigQueryDeleteDataTransferConfigOperator(BaseOperator):
     :type metadata: Optional[Sequence[Tuple[str, str]]]
     :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ("transfer_config_id", "project_id", "gcp_conn_id")
+    template_fields = ("transfer_config_id", "project_id", "gcp_conn_id", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -139,6 +163,7 @@ class BigQueryDeleteDataTransferConfigOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id="google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ):
         super().__init__(**kwargs)
@@ -148,9 +173,13 @@ class BigQueryDeleteDataTransferConfigOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = BiqQueryDataTransferServiceHook(gcp_conn_id=self.gcp_conn_id)
+        hook = BiqQueryDataTransferServiceHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain
+        )
         hook.delete_transfer_config(
             transfer_config_id=self.transfer_config_id,
             project_id=self.project_id,
@@ -196,6 +225,15 @@ class BigQueryDataTransferServiceStartTransferRunsOperator(BaseOperator):
     :type metadata: Optional[Sequence[Tuple[str, str]]]
     :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = (
@@ -204,6 +242,7 @@ class BigQueryDataTransferServiceStartTransferRunsOperator(BaseOperator):
         "requested_time_range",
         "requested_run_time",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -217,6 +256,7 @@ class BigQueryDataTransferServiceStartTransferRunsOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id="google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ):
         super().__init__(**kwargs)
@@ -228,9 +268,13 @@ class BigQueryDataTransferServiceStartTransferRunsOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = BiqQueryDataTransferServiceHook(gcp_conn_id=self.gcp_conn_id)
+        hook = BiqQueryDataTransferServiceHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain
+        )
         self.log.info('Submitting manual transfer for %s', self.transfer_config_id)
         response = hook.start_manual_transfer_runs(
             transfer_config_id=self.transfer_config_id,
diff --git a/airflow/providers/google/cloud/operators/bigtable.py b/airflow/providers/google/cloud/operators/bigtable.py
index da3e16b..13abe1e 100644
--- a/airflow/providers/google/cloud/operators/bigtable.py
+++ b/airflow/providers/google/cloud/operators/bigtable.py
@@ -19,7 +19,7 @@
 This module contains Google Cloud Bigtable operators.
 """
 import enum
-from typing import Dict, Iterable, List, Optional, Union
+from typing import Dict, Iterable, List, Optional, Sequence, Union
 
 import google.api_core.exceptions
 from google.cloud.bigtable.column_family import GarbageCollectionRule
@@ -94,10 +94,20 @@ class BigtableCreateInstanceOperator(BaseOperator, BigtableValidationMixin):
                     If None is not specified, Operator will wait indefinitely.
     :param gcp_conn_id: The connection ID to use to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     REQUIRED_ATTRIBUTES: Iterable[str] = ('instance_id', 'main_cluster_id', 'main_cluster_zone')
-    template_fields: Iterable[str] = ['project_id', 'instance_id', 'main_cluster_id', 'main_cluster_zone']
+    template_fields: Iterable[str] = ['project_id', 'instance_id', 'main_cluster_id',
+                                      'main_cluster_zone', 'impersonation_chain', ]
 
     @apply_defaults
     def __init__(self, *,   # pylint: disable=too-many-arguments
@@ -115,6 +125,7 @@ class BigtableCreateInstanceOperator(BaseOperator, BigtableValidationMixin):
                  cluster_storage_type: Optional[enums.StorageType] = None,
                  timeout: Optional[float] = None,
                  gcp_conn_id: str = 'google_cloud_default',
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         self.project_id = project_id
         self.instance_id = instance_id
@@ -131,10 +142,14 @@ class BigtableCreateInstanceOperator(BaseOperator, BigtableValidationMixin):
         self.timeout = timeout
         self._validate_inputs()
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
         super().__init__(**kwargs)
 
     def execute(self, context):
-        hook = BigtableHook(gcp_conn_id=self.gcp_conn_id)
+        hook = BigtableHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         instance = hook.get_instance(project_id=self.project_id,
                                      instance_id=self.instance_id)
         if instance:
@@ -195,10 +210,19 @@ class BigtableUpdateInstanceOperator(BaseOperator, BigtableValidationMixin):
                     If None is not specified, Operator will wait indefinitely.
     :param gcp_conn_id: The connection ID to use to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     REQUIRED_ATTRIBUTES: Iterable[str] = ['instance_id']
-    template_fields: Iterable[str] = ['project_id', 'instance_id']
+    template_fields: Iterable[str] = ['project_id', 'instance_id', 'impersonation_chain', ]
 
     @apply_defaults
     def __init__(self, *,
@@ -209,6 +233,7 @@ class BigtableUpdateInstanceOperator(BaseOperator, BigtableValidationMixin):
                  instance_labels: Optional[Dict] = None,
                  timeout: Optional[float] = None,
                  gcp_conn_id: str = 'google_cloud_default',
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         self.project_id = project_id
         self.instance_id = instance_id
@@ -218,10 +243,14 @@ class BigtableUpdateInstanceOperator(BaseOperator, BigtableValidationMixin):
         self.timeout = timeout
         self._validate_inputs()
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
         super().__init__(**kwargs)
 
     def execute(self, context):
-        hook = BigtableHook(gcp_conn_id=self.gcp_conn_id)
+        hook = BigtableHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         instance = hook.get_instance(project_id=self.project_id,
                                      instance_id=self.instance_id)
         if not instance:
@@ -261,24 +290,38 @@ class BigtableDeleteInstanceOperator(BaseOperator, BigtableValidationMixin):
     :type project_id: str
     :param gcp_conn_id: The connection ID to use to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
     REQUIRED_ATTRIBUTES = ('instance_id',)  # type: Iterable[str]
-    template_fields = ['project_id', 'instance_id']  # type: Iterable[str]
+    template_fields = ['project_id', 'instance_id', 'impersonation_chain', ]  # type: Iterable[str]
 
     @apply_defaults
     def __init__(self, *,
                  instance_id: str,
                  project_id: Optional[str] = None,
                  gcp_conn_id: str = 'google_cloud_default',
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         self.project_id = project_id
         self.instance_id = instance_id
         self._validate_inputs()
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
         super().__init__(**kwargs)
 
     def execute(self, context):
-        hook = BigtableHook(gcp_conn_id=self.gcp_conn_id)
+        hook = BigtableHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         try:
             hook.delete_instance(project_id=self.project_id,
                                  instance_id=self.instance_id)
@@ -321,9 +364,19 @@ class BigtableCreateTableOperator(BaseOperator, BigtableValidationMixin):
                             :class:`google.cloud.bigtable.column_family.GarbageCollectionRule`
     :param gcp_conn_id: The connection ID to use to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
     REQUIRED_ATTRIBUTES = ('instance_id', 'table_id')  # type: Iterable[str]
-    template_fields = ['project_id', 'instance_id', 'table_id']  # type: Iterable[str]
+    template_fields = ['project_id', 'instance_id', 'table_id',
+                       'impersonation_chain', ]  # type: Iterable[str]
 
     @apply_defaults
     def __init__(self, *,
@@ -333,6 +386,7 @@ class BigtableCreateTableOperator(BaseOperator, BigtableValidationMixin):
                  initial_split_keys: Optional[List] = None,
                  column_families: Optional[Dict[str, GarbageCollectionRule]] = None,
                  gcp_conn_id: str = 'google_cloud_default',
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         self.project_id = project_id
         self.instance_id = instance_id
@@ -341,6 +395,7 @@ class BigtableCreateTableOperator(BaseOperator, BigtableValidationMixin):
         self.column_families = column_families or {}
         self._validate_inputs()
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
         super().__init__(**kwargs)
 
     def _compare_column_families(self, hook, instance):
@@ -367,7 +422,10 @@ class BigtableCreateTableOperator(BaseOperator, BigtableValidationMixin):
         return True
 
     def execute(self, context):
-        hook = BigtableHook(gcp_conn_id=self.gcp_conn_id)
+        hook = BigtableHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         instance = hook.get_instance(project_id=self.project_id, instance_id=self.instance_id)
         if not instance:
             raise AirflowException(
@@ -411,9 +469,19 @@ class BigtableDeleteTableOperator(BaseOperator, BigtableValidationMixin):
     :param app_profile_id: Application profile.
     :param gcp_conn_id: The connection ID to use to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
     REQUIRED_ATTRIBUTES = ('instance_id', 'table_id')  # type: Iterable[str]
-    template_fields = ['project_id', 'instance_id', 'table_id']  # type: Iterable[str]
+    template_fields = ['project_id', 'instance_id', 'table_id',
+                       'impersonation_chain', ]  # type: Iterable[str]
 
     @apply_defaults
     def __init__(self, *,
@@ -422,6 +490,7 @@ class BigtableDeleteTableOperator(BaseOperator, BigtableValidationMixin):
                  project_id: Optional[str] = None,
                  app_profile_id: Optional[str] = None,
                  gcp_conn_id: str = 'google_cloud_default',
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         self.project_id = project_id
         self.instance_id = instance_id
@@ -429,10 +498,14 @@ class BigtableDeleteTableOperator(BaseOperator, BigtableValidationMixin):
         self.app_profile_id = app_profile_id
         self._validate_inputs()
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
         super().__init__(**kwargs)
 
     def execute(self, context):
-        hook = BigtableHook(gcp_conn_id=self.gcp_conn_id)
+        hook = BigtableHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         instance = hook.get_instance(project_id=self.project_id,
                                      instance_id=self.instance_id)
         if not instance:
@@ -476,9 +549,19 @@ class BigtableUpdateClusterOperator(BaseOperator, BigtableValidationMixin):
     :param project_id: Optional, the ID of the GCP project.
     :param gcp_conn_id: The connection ID to use to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
     REQUIRED_ATTRIBUTES = ('instance_id', 'cluster_id', 'nodes')  # type: Iterable[str]
-    template_fields = ['project_id', 'instance_id', 'cluster_id', 'nodes']  # type: Iterable[str]
+    template_fields = ['project_id', 'instance_id', 'cluster_id', 'nodes',
+                       'impersonation_chain', ]  # type: Iterable[str]
 
     @apply_defaults
     def __init__(self, *,
@@ -487,6 +570,7 @@ class BigtableUpdateClusterOperator(BaseOperator, BigtableValidationMixin):
                  nodes: int,
                  project_id: Optional[str] = None,
                  gcp_conn_id: str = 'google_cloud_default',
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         self.project_id = project_id
         self.instance_id = instance_id
@@ -494,10 +578,14 @@ class BigtableUpdateClusterOperator(BaseOperator, BigtableValidationMixin):
         self.nodes = nodes
         self._validate_inputs()
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
         super().__init__(**kwargs)
 
     def execute(self, context):
-        hook = BigtableHook(gcp_conn_id=self.gcp_conn_id)
+        hook = BigtableHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         instance = hook.get_instance(project_id=self.project_id,
                                      instance_id=self.instance_id)
         if not instance:
diff --git a/airflow/providers/google/cloud/operators/cloud_build.py b/airflow/providers/google/cloud/operators/cloud_build.py
index 619330e..159f24e 100644
--- a/airflow/providers/google/cloud/operators/cloud_build.py
+++ b/airflow/providers/google/cloud/operators/cloud_build.py
@@ -19,7 +19,7 @@
 import json
 import re
 from copy import deepcopy
-from typing import Any, Dict, Optional, Union
+from typing import Any, Dict, Optional, Sequence, Union
 from urllib.parse import unquote, urlparse
 
 import yaml
@@ -178,9 +178,18 @@ class CloudBuildCreateBuildOperator(BaseOperator):
     :type gcp_conn_id: str
     :param api_version: API version used (for example v1 or v1beta1).
     :type api_version: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ("body", "gcp_conn_id", "api_version")
+    template_fields = ("body", "gcp_conn_id", "api_version", "impersonation_chain",)
     template_ext = ['.yml', '.yaml', '.json']
 
     @apply_defaults
@@ -189,6 +198,7 @@ class CloudBuildCreateBuildOperator(BaseOperator):
                  project_id: Optional[str] = None,
                  gcp_conn_id: str = "google_cloud_default",
                  api_version: str = "v1",
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         super().__init__(**kwargs)
         self.body = body
@@ -198,6 +208,7 @@ class CloudBuildCreateBuildOperator(BaseOperator):
         self.gcp_conn_id = gcp_conn_id
         self.api_version = api_version
         self._validate_inputs()
+        self.impersonation_chain = impersonation_chain
 
     def prepare_template(self) -> None:
         # if no file is specified, skip
@@ -214,6 +225,10 @@ class CloudBuildCreateBuildOperator(BaseOperator):
             raise AirflowException("The required parameter 'body' is missing")
 
     def execute(self, context):
-        hook = CloudBuildHook(gcp_conn_id=self.gcp_conn_id, api_version=self.api_version)
+        hook = CloudBuildHook(
+            gcp_conn_id=self.gcp_conn_id,
+            api_version=self.api_version,
+            impersonation_chain=self.impersonation_chain
+        )
         body = BuildProcessor(body=self.body).process_body()
         return hook.create_build(body=body, project_id=self.project_id)
diff --git a/airflow/providers/google/cloud/operators/cloud_memorystore.py b/airflow/providers/google/cloud/operators/cloud_memorystore.py
index ebd498a..d28e787 100644
--- a/airflow/providers/google/cloud/operators/cloud_memorystore.py
+++ b/airflow/providers/google/cloud/operators/cloud_memorystore.py
@@ -68,6 +68,15 @@ class CloudMemorystoreCreateInstanceOperator(BaseOperator):
     :type metadata: Sequence[Tuple[str, str]]
     :param gcp_conn_id: The connection ID to use connecting to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = (
@@ -79,6 +88,7 @@ class CloudMemorystoreCreateInstanceOperator(BaseOperator):
         "timeout",
         "metadata",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -92,6 +102,7 @@ class CloudMemorystoreCreateInstanceOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -103,9 +114,13 @@ class CloudMemorystoreCreateInstanceOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context: Dict):
-        hook = CloudMemorystoreHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudMemorystoreHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain
+        )
         result = hook.create_instance(
             location=self.location,
             instance_id=self.instance_id,
@@ -143,9 +158,19 @@ class CloudMemorystoreDeleteInstanceOperator(BaseOperator):
     :type metadata: Sequence[Tuple[str, str]]
     :param gcp_conn_id: The connection ID to use connecting to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ("location", "instance", "project_id", "retry", "timeout", "metadata", "gcp_conn_id")
+    template_fields = ("location", "instance", "project_id", "retry", "timeout", "metadata",
+                       "gcp_conn_id", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -157,6 +182,7 @@ class CloudMemorystoreDeleteInstanceOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -167,9 +193,13 @@ class CloudMemorystoreDeleteInstanceOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context: Dict):
-        hook = CloudMemorystoreHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudMemorystoreHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain
+        )
         hook.delete_instance(
             location=self.location,
             instance=self.instance,
@@ -211,6 +241,15 @@ class CloudMemorystoreExportInstanceOperator(BaseOperator):
     :type metadata: Sequence[Tuple[str, str]]
     :param gcp_conn_id: The connection ID to use connecting to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = (
@@ -222,6 +261,7 @@ class CloudMemorystoreExportInstanceOperator(BaseOperator):
         "timeout",
         "metadata",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -235,6 +275,7 @@ class CloudMemorystoreExportInstanceOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -246,9 +287,13 @@ class CloudMemorystoreExportInstanceOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context: Dict):
-        hook = CloudMemorystoreHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudMemorystoreHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain
+        )
 
         hook.export_instance(
             location=self.location,
@@ -289,6 +334,15 @@ class CloudMemorystoreFailoverInstanceOperator(BaseOperator):
     :type metadata: Sequence[Tuple[str, str]]
     :param gcp_conn_id: The connection ID to use connecting to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = (
@@ -300,6 +354,7 @@ class CloudMemorystoreFailoverInstanceOperator(BaseOperator):
         "timeout",
         "metadata",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -313,6 +368,7 @@ class CloudMemorystoreFailoverInstanceOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -324,9 +380,13 @@ class CloudMemorystoreFailoverInstanceOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context: Dict):
-        hook = CloudMemorystoreHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudMemorystoreHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain
+        )
         hook.failover_instance(
             location=self.location,
             instance=self.instance,
@@ -362,9 +422,19 @@ class CloudMemorystoreGetInstanceOperator(BaseOperator):
     :type metadata: Sequence[Tuple[str, str]]
     :param gcp_conn_id: The connection ID to use connecting to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ("location", "instance", "project_id", "retry", "timeout", "metadata", "gcp_conn_id")
+    template_fields = ("location", "instance", "project_id", "retry", "timeout", "metadata",
+                       "gcp_conn_id", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -376,6 +446,7 @@ class CloudMemorystoreGetInstanceOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -386,9 +457,13 @@ class CloudMemorystoreGetInstanceOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context: Dict):
-        hook = CloudMemorystoreHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudMemorystoreHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain
+        )
         result = hook.get_instance(
             location=self.location,
             instance=self.instance,
@@ -432,6 +507,15 @@ class CloudMemorystoreImportOperator(BaseOperator):
     :type metadata: Sequence[Tuple[str, str]]
     :param gcp_conn_id: The connection ID to use connecting to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = (
@@ -443,6 +527,7 @@ class CloudMemorystoreImportOperator(BaseOperator):
         "timeout",
         "metadata",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -456,6 +541,7 @@ class CloudMemorystoreImportOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -467,9 +553,13 @@ class CloudMemorystoreImportOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context: Dict):
-        hook = CloudMemorystoreHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudMemorystoreHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain
+        )
         hook.import_instance(
             location=self.location,
             instance=self.instance,
@@ -509,9 +599,19 @@ class CloudMemorystoreListInstancesOperator(BaseOperator):
     :type metadata: Sequence[Tuple[str, str]]
     :param gcp_conn_id: The connection ID to use connecting to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ("location", "page_size", "project_id", "retry", "timeout", "metadata", "gcp_conn_id")
+    template_fields = ("location", "page_size", "project_id", "retry", "timeout", "metadata",
+                       "gcp_conn_id", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -523,6 +623,7 @@ class CloudMemorystoreListInstancesOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -533,9 +634,13 @@ class CloudMemorystoreListInstancesOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context: Dict):
-        hook = CloudMemorystoreHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudMemorystoreHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain
+        )
         result = hook.list_instances(
             location=self.location,
             page_size=self.page_size,
@@ -590,6 +695,15 @@ class CloudMemorystoreUpdateInstanceOperator(BaseOperator):
     :type metadata: Sequence[Tuple[str, str]]
     :param gcp_conn_id: The connection ID to use connecting to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = (
@@ -602,6 +716,7 @@ class CloudMemorystoreUpdateInstanceOperator(BaseOperator):
         "timeout",
         "metadata",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -616,6 +731,7 @@ class CloudMemorystoreUpdateInstanceOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -628,9 +744,13 @@ class CloudMemorystoreUpdateInstanceOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context: Dict):
-        hook = CloudMemorystoreHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudMemorystoreHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain
+        )
         hook.update_instance(
             update_mask=self.update_mask,
             instance=self.instance,
@@ -670,6 +790,15 @@ class CloudMemorystoreScaleInstanceOperator(BaseOperator):
     :type metadata: Sequence[Tuple[str, str]]
     :param gcp_conn_id: The connection ID to use connecting to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = (
@@ -681,6 +810,7 @@ class CloudMemorystoreScaleInstanceOperator(BaseOperator):
         "timeout",
         "metadata",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -694,6 +824,7 @@ class CloudMemorystoreScaleInstanceOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -705,9 +836,13 @@ class CloudMemorystoreScaleInstanceOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context: Dict):
-        hook = CloudMemorystoreHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudMemorystoreHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain
+        )
 
         hook.update_instance(
             update_mask={"paths": ["memory_size_gb"]},
@@ -767,6 +902,15 @@ class CloudMemorystoreCreateInstanceAndImportOperator(BaseOperator):
     :type metadata: Sequence[Tuple[str, str]]
     :param gcp_conn_id: The connection ID to use connecting to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = (
@@ -779,6 +923,7 @@ class CloudMemorystoreCreateInstanceAndImportOperator(BaseOperator):
         "timeout",
         "metadata",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -793,6 +938,7 @@ class CloudMemorystoreCreateInstanceAndImportOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -805,9 +951,13 @@ class CloudMemorystoreCreateInstanceAndImportOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context: Dict):
-        hook = CloudMemorystoreHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudMemorystoreHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain
+        )
 
         hook.create_instance(
             location=self.location,
@@ -862,6 +1012,15 @@ class CloudMemorystoreExportAndDeleteInstanceOperator(BaseOperator):
     :type metadata: Sequence[Tuple[str, str]]
     :param gcp_conn_id: The connection ID to use connecting to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = (
@@ -873,6 +1032,7 @@ class CloudMemorystoreExportAndDeleteInstanceOperator(BaseOperator):
         "timeout",
         "metadata",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -886,6 +1046,7 @@ class CloudMemorystoreExportAndDeleteInstanceOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -897,9 +1058,13 @@ class CloudMemorystoreExportAndDeleteInstanceOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context: Dict):
-        hook = CloudMemorystoreHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudMemorystoreHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain
+        )
 
         hook.export_instance(
             location=self.location,
diff --git a/airflow/providers/google/cloud/operators/cloud_sql.py b/airflow/providers/google/cloud/operators/cloud_sql.py
index c755756..ab02455 100644
--- a/airflow/providers/google/cloud/operators/cloud_sql.py
+++ b/airflow/providers/google/cloud/operators/cloud_sql.py
@@ -18,7 +18,7 @@
 """
 This module contains Google Cloud SQL operators.
 """
-from typing import Dict, Iterable, List, Optional, Union
+from typing import Dict, Iterable, List, Optional, Sequence, Union
 
 from googleapiclient.errors import HttpError
 
@@ -152,6 +152,15 @@ class CloudSQLBaseOperator(BaseOperator):
     :type gcp_conn_id: str
     :param api_version: API version used (e.g. v1beta4).
     :type api_version: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
     @apply_defaults
     def __init__(self, *,
@@ -159,11 +168,13 @@ class CloudSQLBaseOperator(BaseOperator):
                  project_id: Optional[str] = None,
                  gcp_conn_id: str = 'google_cloud_default',
                  api_version: str = 'v1beta4',
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         self.project_id = project_id
         self.instance = instance
         self.gcp_conn_id = gcp_conn_id
         self.api_version = api_version
+        self.impersonation_chain = impersonation_chain
         self._validate_inputs()
         super().__init__(**kwargs)
 
@@ -230,9 +241,19 @@ class CloudSQLCreateInstanceOperator(CloudSQLBaseOperator):
     :type api_version: str
     :param validate_body: True if body should be validated, False otherwise.
     :type validate_body: bool
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
     # [START gcp_sql_create_template_fields]
-    template_fields = ('project_id', 'instance', 'body', 'gcp_conn_id', 'api_version')
+    template_fields = ('project_id', 'instance', 'body', 'gcp_conn_id', 'api_version',
+                       'impersonation_chain',)
     # [END gcp_sql_create_template_fields]
 
     @apply_defaults
@@ -243,12 +264,13 @@ class CloudSQLCreateInstanceOperator(CloudSQLBaseOperator):
                  gcp_conn_id: str = 'google_cloud_default',
                  api_version: str = 'v1beta4',
                  validate_body: bool = True,
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         self.body = body
         self.validate_body = validate_body
         super().__init__(
             project_id=project_id, instance=instance, gcp_conn_id=gcp_conn_id,
-            api_version=api_version, **kwargs)
+            api_version=api_version, impersonation_chain=impersonation_chain, **kwargs)
 
     def _validate_inputs(self):
         super()._validate_inputs()
@@ -263,7 +285,8 @@ class CloudSQLCreateInstanceOperator(CloudSQLBaseOperator):
     def execute(self, context):
         hook = CloudSQLHook(
             gcp_conn_id=self.gcp_conn_id,
-            api_version=self.api_version
+            api_version=self.api_version,
+            impersonation_chain=self.impersonation_chain,
         )
         self._validate_body_fields()
         if not self._check_if_instance_exists(self.instance, hook):
@@ -311,9 +334,19 @@ class CloudSQLInstancePatchOperator(CloudSQLBaseOperator):
     :type gcp_conn_id: str
     :param api_version: API version used (e.g. v1beta4).
     :type api_version: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
     # [START gcp_sql_patch_template_fields]
-    template_fields = ('project_id', 'instance', 'body', 'gcp_conn_id', 'api_version')
+    template_fields = ('project_id', 'instance', 'body', 'gcp_conn_id', 'api_version',
+                       'impersonation_chain',)
     # [END gcp_sql_patch_template_fields]
 
     @apply_defaults
@@ -323,11 +356,12 @@ class CloudSQLInstancePatchOperator(CloudSQLBaseOperator):
                  project_id: Optional[str] = None,
                  gcp_conn_id: str = 'google_cloud_default',
                  api_version: str = 'v1beta4',
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         self.body = body
         super().__init__(
             project_id=project_id, instance=instance, gcp_conn_id=gcp_conn_id,
-            api_version=api_version, **kwargs)
+            api_version=api_version, impersonation_chain=impersonation_chain, **kwargs)
 
     def _validate_inputs(self):
         super()._validate_inputs()
@@ -337,7 +371,8 @@ class CloudSQLInstancePatchOperator(CloudSQLBaseOperator):
     def execute(self, context):
         hook = CloudSQLHook(
             gcp_conn_id=self.gcp_conn_id,
-            api_version=self.api_version
+            api_version=self.api_version,
+            impersonation_chain=self.impersonation_chain,
         )
         if not self._check_if_instance_exists(self.instance, hook):
             raise AirflowException('Cloud SQL instance with ID {} does not exist. '
@@ -367,9 +402,19 @@ class CloudSQLDeleteInstanceOperator(CloudSQLBaseOperator):
     :type gcp_conn_id: str
     :param api_version: API version used (e.g. v1beta4).
     :type api_version: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
     # [START gcp_sql_delete_template_fields]
-    template_fields = ('project_id', 'instance', 'gcp_conn_id', 'api_version')
+    template_fields = ('project_id', 'instance', 'gcp_conn_id', 'api_version',
+                       'impersonation_chain',)
     # [END gcp_sql_delete_template_fields]
 
     @apply_defaults
@@ -378,15 +423,17 @@ class CloudSQLDeleteInstanceOperator(CloudSQLBaseOperator):
                  project_id: Optional[str] = None,
                  gcp_conn_id: str = 'google_cloud_default',
                  api_version: str = 'v1beta4',
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         super().__init__(
             project_id=project_id, instance=instance, gcp_conn_id=gcp_conn_id,
-            api_version=api_version, **kwargs)
+            api_version=api_version, impersonation_chain=impersonation_chain, **kwargs)
 
     def execute(self, context):
         hook = CloudSQLHook(
             gcp_conn_id=self.gcp_conn_id,
-            api_version=self.api_version
+            api_version=self.api_version,
+            impersonation_chain=self.impersonation_chain,
         )
         if not self._check_if_instance_exists(self.instance, hook):
             print("Cloud SQL instance with ID {} does not exist. Aborting delete."
@@ -420,9 +467,19 @@ class CloudSQLCreateInstanceDatabaseOperator(CloudSQLBaseOperator):
     :type api_version: str
     :param validate_body: Whether the body should be validated. Defaults to True.
     :type validate_body: bool
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
     # [START gcp_sql_db_create_template_fields]
-    template_fields = ('project_id', 'instance', 'body', 'gcp_conn_id', 'api_version')
+    template_fields = ('project_id', 'instance', 'body', 'gcp_conn_id', 'api_version',
+                       'impersonation_chain',)
     # [END gcp_sql_db_create_template_fields]
 
     @apply_defaults
@@ -433,12 +490,13 @@ class CloudSQLCreateInstanceDatabaseOperator(CloudSQLBaseOperator):
                  gcp_conn_id: str = 'google_cloud_default',
                  api_version: str = 'v1beta4',
                  validate_body: bool = True,
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         self.body = body
         self.validate_body = validate_body
         super().__init__(
             project_id=project_id, instance=instance, gcp_conn_id=gcp_conn_id,
-            api_version=api_version, **kwargs)
+            api_version=api_version, impersonation_chain=impersonation_chain, **kwargs)
 
     def _validate_inputs(self):
         super()._validate_inputs()
@@ -459,7 +517,8 @@ class CloudSQLCreateInstanceDatabaseOperator(CloudSQLBaseOperator):
             return False
         hook = CloudSQLHook(
             gcp_conn_id=self.gcp_conn_id,
-            api_version=self.api_version
+            api_version=self.api_version,
+            impersonation_chain=self.impersonation_chain,
         )
         if self._check_if_db_exists(database, hook):
             self.log.info("Cloud SQL instance with ID %s already contains database"
@@ -498,10 +557,19 @@ class CloudSQLPatchInstanceDatabaseOperator(CloudSQLBaseOperator):
     :type api_version: str
     :param validate_body: Whether the body should be validated. Defaults to True.
     :type validate_body: bool
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
     # [START gcp_sql_db_patch_template_fields]
     template_fields = ('project_id', 'instance', 'body', 'database', 'gcp_conn_id',
-                       'api_version')
+                       'api_version', 'impersonation_chain',)
     # [END gcp_sql_db_patch_template_fields]
 
     @apply_defaults
@@ -513,13 +581,14 @@ class CloudSQLPatchInstanceDatabaseOperator(CloudSQLBaseOperator):
                  gcp_conn_id: str = 'google_cloud_default',
                  api_version: str = 'v1beta4',
                  validate_body: bool = True,
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         self.database = database
         self.body = body
         self.validate_body = validate_body
         super().__init__(
             project_id=project_id, instance=instance, gcp_conn_id=gcp_conn_id,
-            api_version=api_version, **kwargs)
+            api_version=api_version, impersonation_chain=impersonation_chain, **kwargs)
 
     def _validate_inputs(self):
         super()._validate_inputs()
@@ -537,7 +606,8 @@ class CloudSQLPatchInstanceDatabaseOperator(CloudSQLBaseOperator):
         self._validate_body_fields()
         hook = CloudSQLHook(
             gcp_conn_id=self.gcp_conn_id,
-            api_version=self.api_version
+            api_version=self.api_version,
+            impersonation_chain=self.impersonation_chain,
         )
         if not self._check_if_db_exists(self.database, hook):
             raise AirflowException("Cloud SQL instance with ID {instance} does not contain "
@@ -571,10 +641,19 @@ class CloudSQLDeleteInstanceDatabaseOperator(CloudSQLBaseOperator):
     :type gcp_conn_id: str
     :param api_version: API version used (e.g. v1beta4).
     :type api_version: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
     # [START gcp_sql_db_delete_template_fields]
     template_fields = ('project_id', 'instance', 'database', 'gcp_conn_id',
-                       'api_version')
+                       'api_version', 'impersonation_chain',)
     # [END gcp_sql_db_delete_template_fields]
 
     @apply_defaults
@@ -584,11 +663,12 @@ class CloudSQLDeleteInstanceDatabaseOperator(CloudSQLBaseOperator):
                  project_id: Optional[str] = None,
                  gcp_conn_id: str = 'google_cloud_default',
                  api_version: str = 'v1beta4',
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         self.database = database
         super().__init__(
             project_id=project_id, instance=instance, gcp_conn_id=gcp_conn_id,
-            api_version=api_version, **kwargs)
+            api_version=api_version, impersonation_chain=impersonation_chain, **kwargs)
 
     def _validate_inputs(self):
         super()._validate_inputs()
@@ -598,7 +678,8 @@ class CloudSQLDeleteInstanceDatabaseOperator(CloudSQLBaseOperator):
     def execute(self, context):
         hook = CloudSQLHook(
             gcp_conn_id=self.gcp_conn_id,
-            api_version=self.api_version
+            api_version=self.api_version,
+            impersonation_chain=self.impersonation_chain,
         )
         if not self._check_if_db_exists(self.database, hook):
             print("Cloud SQL instance with ID {} does not contain database '{}'. "
@@ -638,9 +719,19 @@ class CloudSQLExportInstanceOperator(CloudSQLBaseOperator):
     :type api_version: str
     :param validate_body: Whether the body should be validated. Defaults to True.
     :type validate_body: bool
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
     # [START gcp_sql_export_template_fields]
-    template_fields = ('project_id', 'instance', 'body', 'gcp_conn_id', 'api_version')
+    template_fields = ('project_id', 'instance', 'body', 'gcp_conn_id', 'api_version',
+                       'impersonation_chain',)
     # [END gcp_sql_export_template_fields]
 
     @apply_defaults
@@ -651,12 +742,13 @@ class CloudSQLExportInstanceOperator(CloudSQLBaseOperator):
                  gcp_conn_id: str = 'google_cloud_default',
                  api_version: str = 'v1beta4',
                  validate_body: bool = True,
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         self.body = body
         self.validate_body = validate_body
         super().__init__(
             project_id=project_id, instance=instance, gcp_conn_id=gcp_conn_id,
-            api_version=api_version, **kwargs)
+            api_version=api_version, impersonation_chain=impersonation_chain, **kwargs)
 
     def _validate_inputs(self):
         super()._validate_inputs()
@@ -672,7 +764,8 @@ class CloudSQLExportInstanceOperator(CloudSQLBaseOperator):
         self._validate_body_fields()
         hook = CloudSQLHook(
             gcp_conn_id=self.gcp_conn_id,
-            api_version=self.api_version
+            api_version=self.api_version,
+            impersonation_chain=self.impersonation_chain,
         )
         return hook.export_instance(
             project_id=self.project_id,
@@ -718,9 +811,19 @@ class CloudSQLImportInstanceOperator(CloudSQLBaseOperator):
     :type api_version: str
     :param validate_body: Whether the body should be validated. Defaults to True.
     :type validate_body: bool
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
     # [START gcp_sql_import_template_fields]
-    template_fields = ('project_id', 'instance', 'body', 'gcp_conn_id', 'api_version')
+    template_fields = ('project_id', 'instance', 'body', 'gcp_conn_id', 'api_version',
+                       'impersonation_chain',)
     # [END gcp_sql_import_template_fields]
 
     @apply_defaults
@@ -731,12 +834,13 @@ class CloudSQLImportInstanceOperator(CloudSQLBaseOperator):
                  gcp_conn_id: str = 'google_cloud_default',
                  api_version: str = 'v1beta4',
                  validate_body: bool = True,
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         self.body = body
         self.validate_body = validate_body
         super().__init__(
             project_id=project_id, instance=instance, gcp_conn_id=gcp_conn_id,
-            api_version=api_version, **kwargs)
+            api_version=api_version, impersonation_chain=impersonation_chain, **kwargs)
 
     def _validate_inputs(self):
         super()._validate_inputs()
@@ -752,7 +856,8 @@ class CloudSQLImportInstanceOperator(CloudSQLBaseOperator):
         self._validate_body_fields()
         hook = CloudSQLHook(
             gcp_conn_id=self.gcp_conn_id,
-            api_version=self.api_version
+            api_version=self.api_version,
+            impersonation_chain=self.impersonation_chain,
         )
         return hook.import_instance(
             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 f2ad319..114366a 100644
--- a/airflow/providers/google/cloud/operators/cloud_storage_transfer_service.py
+++ b/airflow/providers/google/cloud/operators/cloud_storage_transfer_service.py
@@ -21,7 +21,7 @@ This module contains Google Cloud Transfer operators.
 """
 from copy import deepcopy
 from datetime import date, time
-from typing import Dict, Optional
+from typing import Dict, Optional, Sequence, Union
 
 from airflow.exceptions import AirflowException
 from airflow.models import BaseOperator
@@ -190,9 +190,18 @@ class CloudDataTransferServiceCreateJobOperator(BaseOperator):
     :type gcp_conn_id: str
     :param api_version: API version used (e.g. v1).
     :type api_version: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type google_impersonation_chain: Union[str, Sequence[str]]
     """
     # [START gcp_transfer_job_create_template_fields]
-    template_fields = ('body', 'gcp_conn_id', 'aws_conn_id')
+    template_fields = ('body', 'gcp_conn_id', 'aws_conn_id', 'google_impersonation_chain',)
     # [END gcp_transfer_job_create_template_fields]
 
     @apply_defaults
@@ -202,6 +211,7 @@ class CloudDataTransferServiceCreateJobOperator(BaseOperator):
         aws_conn_id: str = 'aws_default',
         gcp_conn_id: str = 'google_cloud_default',
         api_version: str = 'v1',
+        google_impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -209,6 +219,7 @@ class CloudDataTransferServiceCreateJobOperator(BaseOperator):
         self.aws_conn_id = aws_conn_id
         self.gcp_conn_id = gcp_conn_id
         self.api_version = api_version
+        self.google_impersonation_chain = google_impersonation_chain
         self._validate_inputs()
 
     def _validate_inputs(self):
@@ -216,7 +227,11 @@ class CloudDataTransferServiceCreateJobOperator(BaseOperator):
 
     def execute(self, context):
         TransferJobPreprocessor(body=self.body, aws_conn_id=self.aws_conn_id).process_body()
-        hook = CloudDataTransferServiceHook(api_version=self.api_version, gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDataTransferServiceHook(
+            api_version=self.api_version,
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.google_impersonation_chain,
+        )
         return hook.create_transfer_job(body=self.body)
 
 
@@ -248,9 +263,19 @@ class CloudDataTransferServiceUpdateJobOperator(BaseOperator):
     :type gcp_conn_id: str
     :param api_version: API version used (e.g. v1).
     :type api_version: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type google_impersonation_chain: Union[str, Sequence[str]]
     """
     # [START gcp_transfer_job_update_template_fields]
-    template_fields = ('job_name', 'body', 'gcp_conn_id', 'aws_conn_id')
+    template_fields = ('job_name', 'body', 'gcp_conn_id', 'aws_conn_id',
+                       'google_impersonation_chain',)
     # [END gcp_transfer_job_update_template_fields]
 
     @apply_defaults
@@ -261,6 +286,7 @@ class CloudDataTransferServiceUpdateJobOperator(BaseOperator):
         aws_conn_id: str = 'aws_default',
         gcp_conn_id: str = 'google_cloud_default',
         api_version: str = 'v1',
+        google_impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -269,6 +295,7 @@ class CloudDataTransferServiceUpdateJobOperator(BaseOperator):
         self.gcp_conn_id = gcp_conn_id
         self.api_version = api_version
         self.aws_conn_id = aws_conn_id
+        self.google_impersonation_chain = google_impersonation_chain
         self._validate_inputs()
 
     def _validate_inputs(self):
@@ -278,7 +305,11 @@ class CloudDataTransferServiceUpdateJobOperator(BaseOperator):
 
     def execute(self, context):
         TransferJobPreprocessor(body=self.body, aws_conn_id=self.aws_conn_id).process_body()
-        hook = CloudDataTransferServiceHook(api_version=self.api_version, gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDataTransferServiceHook(
+            api_version=self.api_version,
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.google_impersonation_chain,
+        )
         return hook.update_transfer_job(job_name=self.job_name, body=self.body)
 
 
@@ -304,9 +335,19 @@ class CloudDataTransferServiceDeleteJobOperator(BaseOperator):
     :type gcp_conn_id: str
     :param api_version: API version used (e.g. v1).
     :type api_version: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type google_impersonation_chain: Union[str, Sequence[str]]
     """
     # [START gcp_transfer_job_delete_template_fields]
-    template_fields = ('job_name', 'project_id', 'gcp_conn_id', 'api_version')
+    template_fields = ('job_name', 'project_id', 'gcp_conn_id', 'api_version',
+                       'google_impersonation_chain',)
     # [END gcp_transfer_job_delete_template_fields]
 
     @apply_defaults
@@ -316,6 +357,7 @@ class CloudDataTransferServiceDeleteJobOperator(BaseOperator):
         gcp_conn_id: str = "google_cloud_default",
         api_version: str = "v1",
         project_id: Optional[str] = None,
+        google_impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -323,6 +365,7 @@ class CloudDataTransferServiceDeleteJobOperator(BaseOperator):
         self.project_id = project_id
         self.gcp_conn_id = gcp_conn_id
         self.api_version = api_version
+        self.google_impersonation_chain = google_impersonation_chain
         self._validate_inputs()
 
     def _validate_inputs(self):
@@ -331,7 +374,11 @@ class CloudDataTransferServiceDeleteJobOperator(BaseOperator):
 
     def execute(self, context):
         self._validate_inputs()
-        hook = CloudDataTransferServiceHook(api_version=self.api_version, gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDataTransferServiceHook(
+            api_version=self.api_version,
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.google_impersonation_chain,
+        )
         hook.delete_transfer_job(job_name=self.job_name, project_id=self.project_id)
 
 
@@ -351,9 +398,18 @@ class CloudDataTransferServiceGetOperationOperator(BaseOperator):
     :type gcp_conn_id: str
     :param api_version: API version used (e.g. v1).
     :type api_version: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type google_impersonation_chain: Union[str, Sequence[str]]
     """
     # [START gcp_transfer_operation_get_template_fields]
-    template_fields = ('operation_name', 'gcp_conn_id')
+    template_fields = ('operation_name', 'gcp_conn_id', 'google_impersonation_chain',)
     # [END gcp_transfer_operation_get_template_fields]
 
     @apply_defaults
@@ -362,12 +418,14 @@ class CloudDataTransferServiceGetOperationOperator(BaseOperator):
         operation_name: str,
         gcp_conn_id: str = "google_cloud_default",
         api_version: str = "v1",
+        google_impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
         self.operation_name = operation_name
         self.gcp_conn_id = gcp_conn_id
         self.api_version = api_version
+        self.google_impersonation_chain = google_impersonation_chain
         self._validate_inputs()
 
     def _validate_inputs(self):
@@ -375,7 +433,11 @@ class CloudDataTransferServiceGetOperationOperator(BaseOperator):
             raise AirflowException("The required parameter 'operation_name' is empty or None")
 
     def execute(self, context):
-        hook = CloudDataTransferServiceHook(api_version=self.api_version, gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDataTransferServiceHook(
+            api_version=self.api_version,
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.google_impersonation_chain,
+        )
         operation = hook.get_transfer_operation(operation_name=self.operation_name)
         return operation
 
@@ -397,15 +459,25 @@ class CloudDataTransferServiceListOperationsOperator(BaseOperator):
     :type gcp_conn_id: str
     :param api_version: API version used (e.g. v1).
     :type api_version: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type google_impersonation_chain: Union[str, Sequence[str]]
     """
     # [START gcp_transfer_operations_list_template_fields]
-    template_fields = ('filter', 'gcp_conn_id')
+    template_fields = ('filter', 'gcp_conn_id', 'google_impersonation_chain',)
     # [END gcp_transfer_operations_list_template_fields]
 
     def __init__(self,
                  request_filter: Optional[Dict] = None,
                  gcp_conn_id: str = 'google_cloud_default',
                  api_version: str = 'v1',
+                 google_impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         # To preserve backward compatibility
         # TODO: remove one day
@@ -420,6 +492,7 @@ class CloudDataTransferServiceListOperationsOperator(BaseOperator):
         self.filter = request_filter
         self.gcp_conn_id = gcp_conn_id
         self.api_version = api_version
+        self.google_impersonation_chain = google_impersonation_chain
         self._validate_inputs()
 
     def _validate_inputs(self):
@@ -427,7 +500,11 @@ class CloudDataTransferServiceListOperationsOperator(BaseOperator):
             raise AirflowException("The required parameter 'filter' is empty or None")
 
     def execute(self, context):
-        hook = CloudDataTransferServiceHook(api_version=self.api_version, gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDataTransferServiceHook(
+            api_version=self.api_version,
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.google_impersonation_chain,
+        )
         operations_list = hook.list_transfer_operations(request_filter=self.filter)
         self.log.info(operations_list)
         return operations_list
@@ -447,9 +524,19 @@ class CloudDataTransferServicePauseOperationOperator(BaseOperator):
     :type gcp_conn_id: str
     :param api_version:  API version used (e.g. v1).
     :type api_version: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type google_impersonation_chain: Union[str, Sequence[str]]
     """
     # [START gcp_transfer_operation_pause_template_fields]
-    template_fields = ('operation_name', 'gcp_conn_id', 'api_version')
+    template_fields = ('operation_name', 'gcp_conn_id', 'api_version',
+                       'google_impersonation_chain',)
     # [END gcp_transfer_operation_pause_template_fields]
 
     @apply_defaults
@@ -458,12 +545,14 @@ class CloudDataTransferServicePauseOperationOperator(BaseOperator):
         operation_name: str,
         gcp_conn_id: str = "google_cloud_default",
         api_version: str = "v1",
+        google_impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
         self.operation_name = operation_name
         self.gcp_conn_id = gcp_conn_id
         self.api_version = api_version
+        self.google_impersonation_chain = google_impersonation_chain
         self._validate_inputs()
 
     def _validate_inputs(self):
@@ -471,7 +560,11 @@ class CloudDataTransferServicePauseOperationOperator(BaseOperator):
             raise AirflowException("The required parameter 'operation_name' is empty or None")
 
     def execute(self, context):
-        hook = CloudDataTransferServiceHook(api_version=self.api_version, gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDataTransferServiceHook(
+            api_version=self.api_version,
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.google_impersonation_chain,
+        )
         hook.pause_transfer_operation(operation_name=self.operation_name)
 
 
@@ -489,9 +582,19 @@ class CloudDataTransferServiceResumeOperationOperator(BaseOperator):
     :param api_version: API version used (e.g. v1).
     :type api_version: str
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type google_impersonation_chain: Union[str, Sequence[str]]
     """
     # [START gcp_transfer_operation_resume_template_fields]
-    template_fields = ('operation_name', 'gcp_conn_id', 'api_version')
+    template_fields = ('operation_name', 'gcp_conn_id', 'api_version',
+                       'google_impersonation_chain',)
     # [END gcp_transfer_operation_resume_template_fields]
 
     @apply_defaults
@@ -500,11 +603,13 @@ class CloudDataTransferServiceResumeOperationOperator(BaseOperator):
         operation_name: str,
         gcp_conn_id: str = "google_cloud_default",
         api_version: str = "v1",
+        google_impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         self.operation_name = operation_name
         self.gcp_conn_id = gcp_conn_id
         self.api_version = api_version
+        self.google_impersonation_chain = google_impersonation_chain
         self._validate_inputs()
         super().__init__(**kwargs)
 
@@ -513,7 +618,11 @@ class CloudDataTransferServiceResumeOperationOperator(BaseOperator):
             raise AirflowException("The required parameter 'operation_name' is empty or None")
 
     def execute(self, context):
-        hook = CloudDataTransferServiceHook(api_version=self.api_version, gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDataTransferServiceHook(
+            api_version=self.api_version,
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.google_impersonation_chain,
+        )
         hook.resume_transfer_operation(operation_name=self.operation_name)
 
 
@@ -532,9 +641,19 @@ class CloudDataTransferServiceCancelOperationOperator(BaseOperator):
     :param gcp_conn_id: The connection ID used to connect to Google
         Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type google_impersonation_chain: Union[str, Sequence[str]]
     """
     # [START gcp_transfer_operation_cancel_template_fields]
-    template_fields = ('operation_name', 'gcp_conn_id', 'api_version')
+    template_fields = ('operation_name', 'gcp_conn_id', 'api_version',
+                       'google_impersonation_chain',)
     # [END gcp_transfer_operation_cancel_template_fields]
 
     @apply_defaults
@@ -543,12 +662,14 @@ class CloudDataTransferServiceCancelOperationOperator(BaseOperator):
         operation_name: str,
         gcp_conn_id: str = "google_cloud_default",
         api_version: str = "v1",
+        google_impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
         self.operation_name = operation_name
         self.api_version = api_version
         self.gcp_conn_id = gcp_conn_id
+        self.google_impersonation_chain = google_impersonation_chain
         self._validate_inputs()
 
     def _validate_inputs(self):
@@ -556,7 +677,11 @@ class CloudDataTransferServiceCancelOperationOperator(BaseOperator):
             raise AirflowException("The required parameter 'operation_name' is empty or None")
 
     def execute(self, context):
-        hook = CloudDataTransferServiceHook(api_version=self.api_version, gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDataTransferServiceHook(
+            api_version=self.api_version,
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.google_impersonation_chain,
+        )
         hook.cancel_transfer_operation(operation_name=self.operation_name)
 
 
@@ -594,10 +719,9 @@ class CloudDataTransferServiceS3ToGCSOperator(BaseOperator):
     :param gcp_conn_id: The destination connection ID to use
         when connecting to Google Cloud Storage.
     :type gcp_conn_id: str
-    :param delegate_to: The account to impersonate, if any.
-        For this to work, the service account making the request must have
+    :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.
-    :type delegate_to: str
     :param description: Optional transfer service job description
     :type description: str
     :param schedule: Optional transfer service schedule;
@@ -620,9 +744,19 @@ class CloudDataTransferServiceS3ToGCSOperator(BaseOperator):
     :type wait: bool
     :param timeout: Time to wait for the operation to end in seconds. Defaults to 60 seconds if not specified.
     :type timeout: Optional[Union[float, timedelta]]
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type google_impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ('gcp_conn_id', 's3_bucket', 'gcs_bucket', 'description', 'object_conditions')
+    template_fields = ('gcp_conn_id', 's3_bucket', 'gcs_bucket', 'description', 'object_conditions',
+                       'google_impersonation_chain',)
     ui_color = '#e09411'
 
     @apply_defaults
@@ -640,6 +774,7 @@ class CloudDataTransferServiceS3ToGCSOperator(BaseOperator):
         transfer_options: Optional[Dict] = None,
         wait: bool = True,
         timeout: Optional[float] = None,
+        google_impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
 
@@ -656,9 +791,14 @@ class CloudDataTransferServiceS3ToGCSOperator(BaseOperator):
         self.transfer_options = transfer_options
         self.wait = wait
         self.timeout = timeout
+        self.google_impersonation_chain = google_impersonation_chain
 
     def execute(self, context):
-        hook = CloudDataTransferServiceHook(gcp_conn_id=self.gcp_conn_id, delegate_to=self.delegate_to)
+        hook = CloudDataTransferServiceHook(
+            gcp_conn_id=self.gcp_conn_id,
+            delegate_to=self.delegate_to,
+            impersonation_chain=self.google_impersonation_chain,
+        )
         body = self._create_body()
 
         TransferJobPreprocessor(body=body, aws_conn_id=self.aws_conn_id, default_schedule=True).process_body()
@@ -730,10 +870,9 @@ class CloudDataTransferServiceGCSToGCSOperator(BaseOperator):
     :param gcp_conn_id: Optional connection ID to use when connecting to Google Cloud
         Storage.
     :type gcp_conn_id: str
-    :param delegate_to: The account to impersonate, if any.
-        For this to work, the service account making the request must have
+    :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.
-    :type delegate_to: str
     :param description: Optional transfer service job description
     :type description: str
     :param schedule: Optional transfer service schedule;
@@ -756,6 +895,15 @@ class CloudDataTransferServiceGCSToGCSOperator(BaseOperator):
     :type wait: bool
     :param timeout: Time to wait for the operation to end in seconds. Defaults to 60 seconds if not specified.
     :type timeout: Optional[Union[float, timedelta]]
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type google_impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = (
@@ -764,6 +912,7 @@ class CloudDataTransferServiceGCSToGCSOperator(BaseOperator):
         'destination_bucket',
         'description',
         'object_conditions',
+        'google_impersonation_chain',
     )
     ui_color = '#e09411'
 
@@ -781,6 +930,7 @@ class CloudDataTransferServiceGCSToGCSOperator(BaseOperator):
         transfer_options: Optional[Dict] = None,
         wait: bool = True,
         timeout: Optional[float] = None,
+        google_impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
 
@@ -796,9 +946,14 @@ class CloudDataTransferServiceGCSToGCSOperator(BaseOperator):
         self.transfer_options = transfer_options
         self.wait = wait
         self.timeout = timeout
+        self.google_impersonation_chain = google_impersonation_chain
 
     def execute(self, context):
-        hook = CloudDataTransferServiceHook(gcp_conn_id=self.gcp_conn_id, delegate_to=self.delegate_to)
+        hook = CloudDataTransferServiceHook(
+            gcp_conn_id=self.gcp_conn_id,
+            delegate_to=self.delegate_to,
+            impersonation_chain=self.google_impersonation_chain,
+        )
 
         body = self._create_body()
 
diff --git a/airflow/providers/google/cloud/operators/compute.py b/airflow/providers/google/cloud/operators/compute.py
index 33a2472..e3fe98d 100644
--- a/airflow/providers/google/cloud/operators/compute.py
+++ b/airflow/providers/google/cloud/operators/compute.py
@@ -20,7 +20,7 @@ This module contains Google Compute Engine operators.
 """
 
 from copy import deepcopy
-from typing import Any, Dict, List, Optional
+from typing import Any, Dict, List, Optional, Sequence, Union
 
 from googleapiclient.errors import HttpError
 from json_merge_patch import merge
@@ -45,12 +45,14 @@ class ComputeEngineBaseOperator(BaseOperator):
                  project_id: Optional[str] = None,
                  gcp_conn_id: str = 'google_cloud_default',
                  api_version: str = 'v1',
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         self.project_id = project_id
         self.zone = zone
         self.resource_id = resource_id
         self.gcp_conn_id = gcp_conn_id
         self.api_version = api_version
+        self.impersonation_chain = impersonation_chain
         self._validate_inputs()
         super().__init__(**kwargs)
 
@@ -90,9 +92,19 @@ class ComputeEngineStartInstanceOperator(ComputeEngineBaseOperator):
     :type api_version: str
     :param validate_body: Optional, If set to False, body validation is not performed.
         Defaults to False.
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
     # [START gce_instance_start_template_fields]
-    template_fields = ('project_id', 'zone', 'resource_id', 'gcp_conn_id', 'api_version')
+    template_fields = ('project_id', 'zone', 'resource_id', 'gcp_conn_id', 'api_version',
+                       'impersonation_chain',)
     # [END gce_instance_start_template_fields]
 
     @apply_defaults
@@ -102,13 +114,19 @@ class ComputeEngineStartInstanceOperator(ComputeEngineBaseOperator):
                  project_id: Optional[str] = None,
                  gcp_conn_id: str = 'google_cloud_default',
                  api_version: str = 'v1',
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         super().__init__(
             project_id=project_id, zone=zone, resource_id=resource_id,
-            gcp_conn_id=gcp_conn_id, api_version=api_version, **kwargs)
+            gcp_conn_id=gcp_conn_id, api_version=api_version,
+            impersonation_chain=impersonation_chain, **kwargs)
 
     def execute(self, context):
-        hook = ComputeEngineHook(gcp_conn_id=self.gcp_conn_id, api_version=self.api_version)
+        hook = ComputeEngineHook(
+            gcp_conn_id=self.gcp_conn_id,
+            api_version=self.api_version,
+            impersonation_chain=self.impersonation_chain
+        )
         return hook.start_instance(zone=self.zone,
                                    resource_id=self.resource_id,
                                    project_id=self.project_id)
@@ -138,9 +156,19 @@ class ComputeEngineStopInstanceOperator(ComputeEngineBaseOperator):
     :type api_version: str
     :param validate_body: Optional, If set to False, body validation is not performed.
         Defaults to False.
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
     # [START gce_instance_stop_template_fields]
-    template_fields = ('project_id', 'zone', 'resource_id', 'gcp_conn_id', 'api_version')
+    template_fields = ('project_id', 'zone', 'resource_id', 'gcp_conn_id', 'api_version',
+                       'impersonation_chain',)
     # [END gce_instance_stop_template_fields]
 
     @apply_defaults
@@ -150,13 +178,19 @@ class ComputeEngineStopInstanceOperator(ComputeEngineBaseOperator):
                  project_id: Optional[str] = None,
                  gcp_conn_id: str = 'google_cloud_default',
                  api_version: str = 'v1',
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         super().__init__(
             project_id=project_id, zone=zone, resource_id=resource_id,
-            gcp_conn_id=gcp_conn_id, api_version=api_version, **kwargs)
+            gcp_conn_id=gcp_conn_id, api_version=api_version,
+            impersonation_chain=impersonation_chain, **kwargs)
 
     def execute(self, context):
-        hook = ComputeEngineHook(gcp_conn_id=self.gcp_conn_id, api_version=self.api_version)
+        hook = ComputeEngineHook(
+            gcp_conn_id=self.gcp_conn_id,
+            api_version=self.api_version,
+            impersonation_chain=self.impersonation_chain
+        )
         hook.stop_instance(zone=self.zone,
                            resource_id=self.resource_id,
                            project_id=self.project_id)
@@ -196,9 +230,19 @@ class ComputeEngineSetMachineTypeOperator(ComputeEngineBaseOperator):
     :param validate_body: Optional, If set to False, body validation is not performed.
         Defaults to False.
     :type validate_body: bool
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
     # [START gce_instance_set_machine_type_template_fields]
-    template_fields = ('project_id', 'zone', 'resource_id', 'body', 'gcp_conn_id', 'api_version')
+    template_fields = ('project_id', 'zone', 'resource_id', 'body', 'gcp_conn_id', 'api_version',
+                       'impersonation_chain',)
     # [END gce_instance_set_machine_type_template_fields]
 
     @apply_defaults
@@ -210,6 +254,7 @@ class ComputeEngineSetMachineTypeOperator(ComputeEngineBaseOperator):
                  gcp_conn_id: str = 'google_cloud_default',
                  api_version: str = 'v1',
                  validate_body: bool = True,
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         self.body = body
         self._field_validator = None  # type: Optional[GcpBodyFieldValidator]
@@ -218,14 +263,19 @@ class ComputeEngineSetMachineTypeOperator(ComputeEngineBaseOperator):
                 SET_MACHINE_TYPE_VALIDATION_SPECIFICATION, api_version=api_version)
         super().__init__(
             project_id=project_id, zone=zone, resource_id=resource_id,
-            gcp_conn_id=gcp_conn_id, api_version=api_version, **kwargs)
+            gcp_conn_id=gcp_conn_id, api_version=api_version,
+            impersonation_chain=impersonation_chain, **kwargs)
 
     def _validate_all_body_fields(self):
         if self._field_validator:
             self._field_validator.validate(self.body)
 
     def execute(self, context):
-        hook = ComputeEngineHook(gcp_conn_id=self.gcp_conn_id, api_version=self.api_version)
+        hook = ComputeEngineHook(
+            gcp_conn_id=self.gcp_conn_id,
+            api_version=self.api_version,
+            impersonation_chain=self.impersonation_chain
+        )
         self._validate_all_body_fields()
         return hook.set_machine_type(zone=self.zone,
                                      resource_id=self.resource_id,
@@ -317,10 +367,19 @@ class ComputeEngineCopyInstanceTemplateOperator(ComputeEngineBaseOperator):
     :param validate_body: Optional, If set to False, body validation is not performed.
         Defaults to False.
     :type validate_body: bool
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
     # [START gce_instance_template_copy_operator_template_fields]
     template_fields = ('project_id', 'resource_id', 'request_id',
-                       'gcp_conn_id', 'api_version')
+                       'gcp_conn_id', 'api_version', 'impersonation_chain',)
     # [END gce_instance_template_copy_operator_template_fields]
 
     @apply_defaults
@@ -332,6 +391,7 @@ class ComputeEngineCopyInstanceTemplateOperator(ComputeEngineBaseOperator):
                  gcp_conn_id: str = 'google_cloud_default',
                  api_version: str = 'v1',
                  validate_body: bool = True,
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         self.body_patch = body_patch
         self.request_id = request_id
@@ -347,14 +407,19 @@ class ComputeEngineCopyInstanceTemplateOperator(ComputeEngineBaseOperator):
             GCE_INSTANCE_TEMPLATE_FIELDS_TO_SANITIZE)
         super().__init__(
             project_id=project_id, zone='global', resource_id=resource_id,
-            gcp_conn_id=gcp_conn_id, api_version=api_version, **kwargs)
+            gcp_conn_id=gcp_conn_id, api_version=api_version,
+            impersonation_chain=impersonation_chain, **kwargs)
 
     def _validate_all_body_fields(self):
         if self._field_validator:
             self._field_validator.validate(self.body_patch)
 
     def execute(self, context):
-        hook = ComputeEngineHook(gcp_conn_id=self.gcp_conn_id, api_version=self.api_version)
+        hook = ComputeEngineHook(
+            gcp_conn_id=self.gcp_conn_id,
+            api_version=self.api_version,
+            impersonation_chain=self.impersonation_chain
+        )
         self._validate_all_body_fields()
         try:
             # Idempotence check (sort of) - we want to check if the new template
@@ -426,11 +491,20 @@ class ComputeEngineInstanceGroupUpdateManagerTemplateOperator(ComputeEngineBaseO
     :param validate_body: Optional, If set to False, body validation is not performed.
         Defaults to False.
     :type validate_body: bool
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
     # [START gce_igm_update_template_operator_template_fields]
     template_fields = ('project_id', 'resource_id', 'zone', 'request_id',
                        'source_template', 'destination_template',
-                       'gcp_conn_id', 'api_version')
+                       'gcp_conn_id', 'api_version', 'impersonation_chain',)
     # [END gce_igm_update_template_operator_template_fields]
 
     @apply_defaults
@@ -444,6 +518,7 @@ class ComputeEngineInstanceGroupUpdateManagerTemplateOperator(ComputeEngineBaseO
                  request_id: Optional[str] = None,
                  gcp_conn_id: str = 'google_cloud_default',
                  api_version='beta',
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         self.zone = zone
         self.source_template = source_template
@@ -457,7 +532,8 @@ class ComputeEngineInstanceGroupUpdateManagerTemplateOperator(ComputeEngineBaseO
                                    " api version or above")
         super().__init__(
             project_id=project_id, zone=self.zone, resource_id=resource_id,
-            gcp_conn_id=gcp_conn_id, api_version=api_version, **kwargs)
+            gcp_conn_id=gcp_conn_id, api_version=api_version,
+            impersonation_chain=impersonation_chain, **kwargs)
 
     def _possibly_replace_template(self, dictionary: Dict) -> None:
         if dictionary.get('instanceTemplate') == self.source_template:
@@ -465,7 +541,11 @@ class ComputeEngineInstanceGroupUpdateManagerTemplateOperator(ComputeEngineBaseO
             self._change_performed = True
 
     def execute(self, context):
-        hook = ComputeEngineHook(gcp_conn_id=self.gcp_conn_id, api_version=self.api_version)
+        hook = ComputeEngineHook(
+            gcp_conn_id=self.gcp_conn_id,
+            api_version=self.api_version,
+            impersonation_chain=self.impersonation_chain
+        )
         old_instance_group_manager = hook.get_instance_group_manager(
             zone=self.zone, resource_id=self.resource_id, project_id=self.project_id)
         patch_body = {}
diff --git a/airflow/providers/google/cloud/operators/datacatalog.py b/airflow/providers/google/cloud/operators/datacatalog.py
index e8d0398..b8404f9 100644
--- a/airflow/providers/google/cloud/operators/datacatalog.py
+++ b/airflow/providers/google/cloud/operators/datacatalog.py
@@ -66,6 +66,15 @@ class CloudDataCatalogCreateEntryOperator(BaseOperator):
     :param gcp_conn_id: Optional, The connection ID used to connect to Google Cloud
         Platform. Defaults to 'google_cloud_default'.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = (
@@ -78,6 +87,7 @@ class CloudDataCatalogCreateEntryOperator(BaseOperator):
         "timeout",
         "metadata",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -92,6 +102,7 @@ class CloudDataCatalogCreateEntryOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -104,9 +115,13 @@ class CloudDataCatalogCreateEntryOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context: Dict):
-        hook = CloudDataCatalogHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDataCatalogHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain
+        )
         try:
             result = hook.create_entry(
                 location=self.location,
@@ -170,6 +185,15 @@ class CloudDataCatalogCreateEntryGroupOperator(BaseOperator):
     :param gcp_conn_id: Optional, The connection ID used to connect to Google Cloud
         Platform. Defaults to 'google_cloud_default'.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = (
@@ -181,6 +205,7 @@ class CloudDataCatalogCreateEntryGroupOperator(BaseOperator):
         "timeout",
         "metadata",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -194,6 +219,7 @@ class CloudDataCatalogCreateEntryGroupOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -205,9 +231,13 @@ class CloudDataCatalogCreateEntryGroupOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context: Dict):
-        hook = CloudDataCatalogHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDataCatalogHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain
+        )
         try:
             result = hook.create_entry_group(
                 location=self.location,
@@ -271,6 +301,15 @@ class CloudDataCatalogCreateTagOperator(BaseOperator):
     :param gcp_conn_id: Optional, The connection ID used to connect to Google Cloud
         Platform. Defaults to 'google_cloud_default'.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = (
@@ -284,6 +323,7 @@ class CloudDataCatalogCreateTagOperator(BaseOperator):
         "timeout",
         "metadata",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -299,6 +339,7 @@ class CloudDataCatalogCreateTagOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -312,9 +353,13 @@ class CloudDataCatalogCreateTagOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context: Dict):
-        hook = CloudDataCatalogHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDataCatalogHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain
+        )
         try:
             tag = hook.create_tag(
                 location=self.location,
@@ -389,6 +434,15 @@ class CloudDataCatalogCreateTagTemplateOperator(BaseOperator):
     :param gcp_conn_id: Optional, The connection ID used to connect to Google Cloud
         Platform. Defaults to 'google_cloud_default'.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = (
@@ -400,6 +454,7 @@ class CloudDataCatalogCreateTagTemplateOperator(BaseOperator):
         "timeout",
         "metadata",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -413,6 +468,7 @@ class CloudDataCatalogCreateTagTemplateOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -424,9 +480,13 @@ class CloudDataCatalogCreateTagTemplateOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context: Dict):
-        hook = CloudDataCatalogHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDataCatalogHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain
+        )
         try:
             result = hook.create_tag_template(
                 location=self.location,
@@ -491,6 +551,15 @@ class CloudDataCatalogCreateTagTemplateFieldOperator(BaseOperator):
     :param gcp_conn_id: Optional, The connection ID used to connect to Google Cloud
         Platform. Defaults to 'google_cloud_default'.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = (
@@ -503,6 +572,7 @@ class CloudDataCatalogCreateTagTemplateFieldOperator(BaseOperator):
         "timeout",
         "metadata",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -517,6 +587,7 @@ class CloudDataCatalogCreateTagTemplateFieldOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -529,9 +600,13 @@ class CloudDataCatalogCreateTagTemplateFieldOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context: Dict):
-        hook = CloudDataCatalogHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDataCatalogHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain
+        )
         try:
             result = hook.create_tag_template_field(
                 location=self.location,
@@ -588,6 +663,15 @@ class CloudDataCatalogDeleteEntryOperator(BaseOperator):
     :param gcp_conn_id: Optional, The connection ID used to connect to Google Cloud
         Platform. Defaults to 'google_cloud_default'.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = (
@@ -599,6 +683,7 @@ class CloudDataCatalogDeleteEntryOperator(BaseOperator):
         "timeout",
         "metadata",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -612,6 +697,7 @@ class CloudDataCatalogDeleteEntryOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -623,9 +709,13 @@ class CloudDataCatalogDeleteEntryOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context: Dict):
-        hook = CloudDataCatalogHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDataCatalogHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain
+        )
         try:
             hook.delete_entry(
                 location=self.location,
@@ -668,9 +758,19 @@ class CloudDataCatalogDeleteEntryGroupOperator(BaseOperator):
     :param gcp_conn_id: Optional, The connection ID used to connect to Google Cloud
         Platform. Defaults to 'google_cloud_default'.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ("location", "entry_group", "project_id", "retry", "timeout", "metadata", "gcp_conn_id")
+    template_fields = ("location", "entry_group", "project_id", "retry", "timeout", "metadata",
+                       "gcp_conn_id", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -682,6 +782,7 @@ class CloudDataCatalogDeleteEntryGroupOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -692,9 +793,13 @@ class CloudDataCatalogDeleteEntryGroupOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context: Dict):
-        hook = CloudDataCatalogHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDataCatalogHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain
+        )
         try:
             hook.delete_entry_group(
                 location=self.location,
@@ -738,6 +843,15 @@ class CloudDataCatalogDeleteTagOperator(BaseOperator):
     :param gcp_conn_id: Optional, The connection ID used to connect to Google Cloud
         Platform. Defaults to 'google_cloud_default'.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = (
@@ -750,6 +864,7 @@ class CloudDataCatalogDeleteTagOperator(BaseOperator):
         "timeout",
         "metadata",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -764,6 +879,7 @@ class CloudDataCatalogDeleteTagOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -776,9 +892,13 @@ class CloudDataCatalogDeleteTagOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context: Dict):
-        hook = CloudDataCatalogHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDataCatalogHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain
+        )
         try:
             hook.delete_tag(
                 location=self.location,
@@ -824,6 +944,15 @@ class CloudDataCatalogDeleteTagTemplateOperator(BaseOperator):
     :param gcp_conn_id: Optional, The connection ID used to connect to Google Cloud
         Platform. Defaults to 'google_cloud_default'.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = (
@@ -835,6 +964,7 @@ class CloudDataCatalogDeleteTagTemplateOperator(BaseOperator):
         "timeout",
         "metadata",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -848,6 +978,7 @@ class CloudDataCatalogDeleteTagTemplateOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -859,9 +990,13 @@ class CloudDataCatalogDeleteTagTemplateOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context: Dict):
-        hook = CloudDataCatalogHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDataCatalogHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain
+        )
         try:
             hook.delete_tag_template(
                 location=self.location,
@@ -906,6 +1041,15 @@ class CloudDataCatalogDeleteTagTemplateFieldOperator(BaseOperator):
     :param gcp_conn_id: Optional, The connection ID used to connect to Google Cloud
         Platform. Defaults to 'google_cloud_default'.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = (
@@ -918,6 +1062,7 @@ class CloudDataCatalogDeleteTagTemplateFieldOperator(BaseOperator):
         "timeout",
         "metadata",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -932,6 +1077,7 @@ class CloudDataCatalogDeleteTagTemplateFieldOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -944,9 +1090,13 @@ class CloudDataCatalogDeleteTagTemplateFieldOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context: Dict):
-        hook = CloudDataCatalogHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDataCatalogHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain
+        )
         try:
             hook.delete_tag_template_field(
                 location=self.location,
@@ -990,6 +1140,15 @@ class CloudDataCatalogGetEntryOperator(BaseOperator):
     :param gcp_conn_id: Optional, The connection ID used to connect to Google Cloud
         Platform. Defaults to 'google_cloud_default'.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = (
@@ -1001,6 +1160,7 @@ class CloudDataCatalogGetEntryOperator(BaseOperator):
         "timeout",
         "metadata",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -1014,6 +1174,7 @@ class CloudDataCatalogGetEntryOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -1025,9 +1186,13 @@ class CloudDataCatalogGetEntryOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context: Dict):
-        hook = CloudDataCatalogHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDataCatalogHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain
+        )
         result = hook.get_entry(
             location=self.location,
             entry_group=self.entry_group,
@@ -1071,6 +1236,15 @@ class CloudDataCatalogGetEntryGroupOperator(BaseOperator):
     :param gcp_conn_id: Optional, The connection ID used to connect to Google Cloud
         Platform. Defaults to 'google_cloud_default'.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = (
@@ -1082,6 +1256,7 @@ class CloudDataCatalogGetEntryGroupOperator(BaseOperator):
         "timeout",
         "metadata",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -1095,6 +1270,7 @@ class CloudDataCatalogGetEntryGroupOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -1106,9 +1282,13 @@ class CloudDataCatalogGetEntryGroupOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context: Dict):
-        hook = CloudDataCatalogHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDataCatalogHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain
+        )
         result = hook.get_entry_group(
             location=self.location,
             entry_group=self.entry_group,
@@ -1147,6 +1327,15 @@ class CloudDataCatalogGetTagTemplateOperator(BaseOperator):
     :param gcp_conn_id: Optional, The connection ID used to connect to Google Cloud
         Platform. Defaults to 'google_cloud_default'.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = (
@@ -1157,6 +1346,7 @@ class CloudDataCatalogGetTagTemplateOperator(BaseOperator):
         "timeout",
         "metadata",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -1169,6 +1359,7 @@ class CloudDataCatalogGetTagTemplateOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -1179,9 +1370,13 @@ class CloudDataCatalogGetTagTemplateOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context: Dict):
-        hook = CloudDataCatalogHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDataCatalogHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain
+        )
         result = hook.get_tag_template(
             location=self.location,
             tag_template=self.tag_template,
@@ -1225,6 +1420,15 @@ class CloudDataCatalogListTagsOperator(BaseOperator):
     :param gcp_conn_id: Optional, The connection ID used to connect to Google Cloud
         Platform. Defaults to 'google_cloud_default'.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = (
@@ -1237,6 +1441,7 @@ class CloudDataCatalogListTagsOperator(BaseOperator):
         "timeout",
         "metadata",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -1251,6 +1456,7 @@ class CloudDataCatalogListTagsOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -1263,9 +1469,13 @@ class CloudDataCatalogListTagsOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context: Dict):
-        hook = CloudDataCatalogHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDataCatalogHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain
+        )
         result = hook.list_tags(
             location=self.location,
             entry_group=self.entry_group,
@@ -1307,6 +1517,15 @@ class CloudDataCatalogLookupEntryOperator(BaseOperator):
     :param gcp_conn_id: Optional, The connection ID used to connect to Google Cloud
         Platform. Defaults to 'google_cloud_default'.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = (
@@ -1317,6 +1536,7 @@ class CloudDataCatalogLookupEntryOperator(BaseOperator):
         "timeout",
         "metadata",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -1329,6 +1549,7 @@ class CloudDataCatalogLookupEntryOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -1339,9 +1560,13 @@ class CloudDataCatalogLookupEntryOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context: Dict):
-        hook = CloudDataCatalogHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDataCatalogHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain
+        )
         result = hook.lookup_entry(
             linked_resource=self.linked_resource,
             sql_resource=self.sql_resource,
@@ -1384,6 +1609,15 @@ class CloudDataCatalogRenameTagTemplateFieldOperator(BaseOperator):
     :param gcp_conn_id: Optional, The connection ID used to connect to Google Cloud
         Platform. Defaults to 'google_cloud_default'.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = (
@@ -1396,6 +1630,7 @@ class CloudDataCatalogRenameTagTemplateFieldOperator(BaseOperator):
         "timeout",
         "metadata",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -1410,6 +1645,7 @@ class CloudDataCatalogRenameTagTemplateFieldOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -1422,9 +1658,13 @@ class CloudDataCatalogRenameTagTemplateFieldOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context: Dict):
-        hook = CloudDataCatalogHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDataCatalogHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain
+        )
         hook.rename_tag_template_field(
             location=self.location,
             tag_template=self.tag_template,
@@ -1492,6 +1732,15 @@ class CloudDataCatalogSearchCatalogOperator(BaseOperator):
     :param gcp_conn_id: Optional, The connection ID used to connect to Google Cloud
         Platform. Defaults to 'google_cloud_default'.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = (
@@ -1503,6 +1752,7 @@ class CloudDataCatalogSearchCatalogOperator(BaseOperator):
         "timeout",
         "metadata",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -1516,6 +1766,7 @@ class CloudDataCatalogSearchCatalogOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -1527,9 +1778,13 @@ class CloudDataCatalogSearchCatalogOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context: Dict):
-        hook = CloudDataCatalogHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDataCatalogHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain
+        )
         result = hook.search_catalog(
             scope=self.scope,
             query=self.query,
@@ -1581,6 +1836,15 @@ class CloudDataCatalogUpdateEntryOperator(BaseOperator):
     :param gcp_conn_id: Optional, The connection ID used to connect to Google Cloud
         Platform. Defaults to 'google_cloud_default'.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = (
@@ -1594,6 +1858,7 @@ class CloudDataCatalogUpdateEntryOperator(BaseOperator):
         "timeout",
         "metadata",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -1609,6 +1874,7 @@ class CloudDataCatalogUpdateEntryOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -1622,9 +1888,13 @@ class CloudDataCatalogUpdateEntryOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context: Dict):
-        hook = CloudDataCatalogHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDataCatalogHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain
+        )
         hook.update_entry(
             entry=self.entry,
             update_mask=self.update_mask,
@@ -1677,6 +1947,15 @@ class CloudDataCatalogUpdateTagOperator(BaseOperator):
     :param gcp_conn_id: Optional, The connection ID used to connect to Google Cloud
         Platform. Defaults to 'google_cloud_default'.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = (
@@ -1691,6 +1970,7 @@ class CloudDataCatalogUpdateTagOperator(BaseOperator):
         "timeout",
         "metadata",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -1707,6 +1987,7 @@ class CloudDataCatalogUpdateTagOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -1721,9 +2002,13 @@ class CloudDataCatalogUpdateTagOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context: Dict):
-        hook = CloudDataCatalogHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDataCatalogHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain
+        )
         hook.update_tag(
             tag=self.tag,
             update_mask=self.update_mask,
@@ -1780,6 +2065,15 @@ class CloudDataCatalogUpdateTagTemplateOperator(BaseOperator):
     :param gcp_conn_id: Optional, The connection ID used to connect to Google Cloud
         Platform. Defaults to 'google_cloud_default'.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = (
@@ -1792,6 +2086,7 @@ class CloudDataCatalogUpdateTagTemplateOperator(BaseOperator):
         "timeout",
         "metadata",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -1806,6 +2101,7 @@ class CloudDataCatalogUpdateTagTemplateOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -1818,9 +2114,13 @@ class CloudDataCatalogUpdateTagTemplateOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context: Dict):
-        hook = CloudDataCatalogHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDataCatalogHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain
+        )
         hook.update_tag_template(
             tag_template=self.tag_template,
             update_mask=self.update_mask,
@@ -1881,6 +2181,15 @@ class CloudDataCatalogUpdateTagTemplateFieldOperator(BaseOperator):
     :param gcp_conn_id: Optional, The connection ID used to connect to Google Cloud
         Platform. Defaults to 'google_cloud_default'.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = (
@@ -1895,6 +2204,7 @@ class CloudDataCatalogUpdateTagTemplateFieldOperator(BaseOperator):
         "timeout",
         "metadata",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -1911,6 +2221,7 @@ class CloudDataCatalogUpdateTagTemplateFieldOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -1925,9 +2236,13 @@ class CloudDataCatalogUpdateTagTemplateFieldOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context: Dict):
-        hook = CloudDataCatalogHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDataCatalogHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain
+        )
         hook.update_tag_template_field(
             tag_template_field=self.tag_template_field,
             update_mask=self.update_mask,
diff --git a/airflow/providers/google/cloud/operators/dataflow.py b/airflow/providers/google/cloud/operators/dataflow.py
index 33adbac..1d0ce32 100644
--- a/airflow/providers/google/cloud/operators/dataflow.py
+++ b/airflow/providers/google/cloud/operators/dataflow.py
@@ -23,7 +23,7 @@ import copy
 import re
 from contextlib import ExitStack
 from enum import Enum
-from typing import Any, Dict, List, Optional
+from typing import Any, Dict, List, Optional, Sequence, Union
 
 from airflow.models import BaseOperator
 from airflow.providers.google.cloud.hooks.dataflow import DEFAULT_DATAFLOW_LOCATION, DataflowHook
@@ -116,8 +116,8 @@ class DataflowCreateJavaJobOperator(BaseOperator):
     :param gcp_conn_id: The connection ID to use connecting to Google Cloud
         Platform.
     :type gcp_conn_id: str
-    :param delegate_to: The account to impersonate, if any.
-        For this to work, the service account making the request must have
+    :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.
     :type delegate_to: str
     :param poll_sleep: The time in seconds to sleep between polling Google
@@ -295,14 +295,23 @@ class DataflowTemplatedJobStartOperator(BaseOperator):
     :param gcp_conn_id: The connection ID to use connecting to Google Cloud
         Platform.
     :type gcp_conn_id: str
-    :param delegate_to: The account to impersonate, if any.
-        For this to work, the service account making the request must have
+    :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.
     :type delegate_to: str
     :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.
     :type poll_sleep: int
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
 
     It's a good practice to define dataflow_* parameters in the default_args of the dag
     like the project, zone and staging location.
@@ -357,7 +366,8 @@ class DataflowTemplatedJobStartOperator(BaseOperator):
         'parameters',
         'project_id',
         'location',
-        'gcp_conn_id'
+        'gcp_conn_id',
+        'impersonation_chain',
     ]
     ui_color = '#0273d4'
 
@@ -374,6 +384,7 @@ class DataflowTemplatedJobStartOperator(BaseOperator):
             gcp_conn_id: str = 'google_cloud_default',
             delegate_to: Optional[str] = None,
             poll_sleep: int = 10,
+            impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
             **kwargs) -> None:
         super().__init__(**kwargs)
         self.template = template
@@ -388,12 +399,14 @@ class DataflowTemplatedJobStartOperator(BaseOperator):
         self.poll_sleep = poll_sleep
         self.job_id = None
         self.hook: Optional[DataflowHook] = None
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
         self.hook = DataflowHook(
             gcp_conn_id=self.gcp_conn_id,
             delegate_to=self.delegate_to,
-            poll_sleep=self.poll_sleep
+            poll_sleep=self.poll_sleep,
+            impersonation_chain=self.impersonation_chain,
         )
 
         def set_current_job_id(job_id):
@@ -478,9 +491,9 @@ class DataflowCreatePythonJobOperator(BaseOperator):
     :type project_id: str
     :param location: Job location.
     :type location: str
-    :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 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.
     :type delegate_to: str
     :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
diff --git a/airflow/providers/google/cloud/operators/datafusion.py b/airflow/providers/google/cloud/operators/datafusion.py
index 4be569f..4923ea8 100644
--- a/airflow/providers/google/cloud/operators/datafusion.py
+++ b/airflow/providers/google/cloud/operators/datafusion.py
@@ -19,7 +19,7 @@
 This module contains Google DataFusion operators.
 """
 from time import sleep
-from typing import Any, Dict, List, Optional
+from typing import Any, Dict, List, Optional, Sequence, Union
 
 from google.api_core.retry import exponential_sleep_generator
 from googleapiclient.errors import HttpError
@@ -48,12 +48,22 @@ class CloudDataFusionRestartInstanceOperator(BaseOperator):
     :type api_version: str
     :param gcp_conn_id: The connection ID to use when fetching connection info.
     :type gcp_conn_id: str
-    :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 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.
     :type delegate_to: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ("instance_name",)
+    template_fields = ("instance_name", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -64,6 +74,7 @@ class CloudDataFusionRestartInstanceOperator(BaseOperator):
         api_version: str = "v1beta1",
         gcp_conn_id: str = "google_cloud_default",
         delegate_to: Optional[str] = None,
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -73,12 +84,14 @@ class CloudDataFusionRestartInstanceOperator(BaseOperator):
         self.api_version = api_version
         self.gcp_conn_id = gcp_conn_id
         self.delegate_to = delegate_to
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, 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,
         )
         self.log.info("Restarting Data Fusion instance: %s", self.instance_name)
         operation = hook.restart_instance(
@@ -108,12 +121,22 @@ class CloudDataFusionDeleteInstanceOperator(BaseOperator):
     :type api_version: str
     :param gcp_conn_id: The connection ID to use when fetching connection info.
     :type gcp_conn_id: str
-    :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 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.
     :type delegate_to: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ("instance_name",)
+    template_fields = ("instance_name", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -124,6 +147,7 @@ class CloudDataFusionDeleteInstanceOperator(BaseOperator):
         api_version: str = "v1beta1",
         gcp_conn_id: str = "google_cloud_default",
         delegate_to: Optional[str] = None,
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -133,12 +157,14 @@ class CloudDataFusionDeleteInstanceOperator(BaseOperator):
         self.api_version = api_version
         self.gcp_conn_id = gcp_conn_id
         self.delegate_to = delegate_to
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, 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,
         )
         self.log.info("Deleting Data Fusion instance: %s", self.instance_name)
         operation = hook.delete_instance(
@@ -171,12 +197,22 @@ class CloudDataFusionCreateInstanceOperator(BaseOperator):
     :type api_version: str
     :param gcp_conn_id: The connection ID to use when fetching connection info.
     :type gcp_conn_id: str
-    :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 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.
     :type delegate_to: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ("instance_name", "instance")
+    template_fields = ("instance_name", "instance", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -188,6 +224,7 @@ class CloudDataFusionCreateInstanceOperator(BaseOperator):
         api_version: str = "v1beta1",
         gcp_conn_id: str = "google_cloud_default",
         delegate_to: Optional[str] = None,
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -198,12 +235,14 @@ class CloudDataFusionCreateInstanceOperator(BaseOperator):
         self.api_version = api_version
         self.gcp_conn_id = gcp_conn_id
         self.delegate_to = delegate_to
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, 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,
         )
         self.log.info("Creating Data Fusion instance: %s", self.instance_name)
         try:
@@ -261,12 +300,22 @@ class CloudDataFusionUpdateInstanceOperator(BaseOperator):
     :type api_version: str
     :param gcp_conn_id: The connection ID to use when fetching connection info.
     :type gcp_conn_id: str
-    :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 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.
     :type delegate_to: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ("instance_name", "instance")
+    template_fields = ("instance_name", "instance", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -279,6 +328,7 @@ class CloudDataFusionUpdateInstanceOperator(BaseOperator):
         api_version: str = "v1beta1",
         gcp_conn_id: str = "google_cloud_default",
         delegate_to: Optional[str] = None,
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -290,12 +340,14 @@ class CloudDataFusionUpdateInstanceOperator(BaseOperator):
         self.api_version = api_version
         self.gcp_conn_id = gcp_conn_id
         self.delegate_to = delegate_to
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, 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,
         )
         self.log.info("Updating Data Fusion instance: %s", self.instance_name)
         operation = hook.patch_instance(
@@ -327,12 +379,22 @@ class CloudDataFusionGetInstanceOperator(BaseOperator):
     :type api_version: str
     :param gcp_conn_id: The connection ID to use when fetching connection info.
     :type gcp_conn_id: str
-    :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 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.
     :type delegate_to: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ("instance_name",)
+    template_fields = ("instance_name", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -343,6 +405,7 @@ class CloudDataFusionGetInstanceOperator(BaseOperator):
         api_version: str = "v1beta1",
         gcp_conn_id: str = "google_cloud_default",
         delegate_to: Optional[str] = None,
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -352,12 +415,14 @@ class CloudDataFusionGetInstanceOperator(BaseOperator):
         self.api_version = api_version
         self.gcp_conn_id = gcp_conn_id
         self.delegate_to = delegate_to
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, 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,
         )
         self.log.info("Retrieving Data Fusion instance: %s", self.instance_name)
         instance = hook.get_instance(
@@ -393,12 +458,22 @@ class CloudDataFusionCreatePipelineOperator(BaseOperator):
     :type api_version: str
     :param gcp_conn_id: The connection ID to use when fetching connection info.
     :type gcp_conn_id: str
-    :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 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.
     :type delegate_to: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ("instance_name", "pipeline_name")
+    template_fields = ("instance_name", "pipeline_name", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -412,6 +487,7 @@ class CloudDataFusionCreatePipelineOperator(BaseOperator):
         api_version: str = "v1beta1",
         gcp_conn_id: str = "google_cloud_default",
         delegate_to: Optional[str] = None,
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -424,12 +500,14 @@ class CloudDataFusionCreatePipelineOperator(BaseOperator):
         self.api_version = api_version
         self.gcp_conn_id = gcp_conn_id
         self.delegate_to = delegate_to
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, 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,
         )
         self.log.info("Creating Data Fusion pipeline: %s", self.pipeline_name)
         instance = hook.get_instance(
@@ -471,12 +549,22 @@ class CloudDataFusionDeletePipelineOperator(BaseOperator):
     :type api_version: str
     :param gcp_conn_id: The connection ID to use when fetching connection info.
     :type gcp_conn_id: str
-    :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 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.
     :type delegate_to: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ("instance_name", "version_id", "pipeline_name")
+    template_fields = ("instance_name", "version_id", "pipeline_name", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -490,6 +578,7 @@ class CloudDataFusionDeletePipelineOperator(BaseOperator):
         api_version: str = "v1beta1",
         gcp_conn_id: str = "google_cloud_default",
         delegate_to: Optional[str] = None,
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -502,12 +591,14 @@ class CloudDataFusionDeletePipelineOperator(BaseOperator):
         self.api_version = api_version
         self.gcp_conn_id = gcp_conn_id
         self.delegate_to = delegate_to
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, 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,
         )
         self.log.info("Deleting Data Fusion pipeline: %s", self.pipeline_name)
         instance = hook.get_instance(
@@ -550,12 +641,22 @@ class CloudDataFusionListPipelinesOperator(BaseOperator):
     :type api_version: str
     :param gcp_conn_id: The connection ID to use when fetching connection info.
     :type gcp_conn_id: str
-    :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 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.
     :type delegate_to: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ("instance_name", "artifact_name", "artifact_version")
+    template_fields = ("instance_name", "artifact_name", "artifact_version", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -569,6 +670,7 @@ class CloudDataFusionListPipelinesOperator(BaseOperator):
         api_version: str = "v1beta1",
         gcp_conn_id: str = "google_cloud_default",
         delegate_to: Optional[str] = None,
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -581,12 +683,14 @@ class CloudDataFusionListPipelinesOperator(BaseOperator):
         self.api_version = api_version
         self.gcp_conn_id = gcp_conn_id
         self.delegate_to = delegate_to
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, 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,
         )
         self.log.info("Listing Data Fusion pipelines")
         instance = hook.get_instance(
@@ -635,12 +739,22 @@ class CloudDataFusionStartPipelineOperator(BaseOperator):
     :type api_version: str
     :param gcp_conn_id: The connection ID to use when fetching connection info.
     :type gcp_conn_id: str
-    :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 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.
     :type delegate_to: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ("instance_name", "pipeline_name", "runtime_args")
+    template_fields = ("instance_name", "pipeline_name", "runtime_args", "impersonation_chain",)
 
     @apply_defaults
     def __init__(  # pylint: disable=too-many-arguments
@@ -656,6 +770,7 @@ class CloudDataFusionStartPipelineOperator(BaseOperator):
         api_version: str = "v1beta1",
         gcp_conn_id: str = "google_cloud_default",
         delegate_to: Optional[str] = None,
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -670,12 +785,14 @@ class CloudDataFusionStartPipelineOperator(BaseOperator):
         self.api_version = api_version
         self.gcp_conn_id = gcp_conn_id
         self.delegate_to = delegate_to
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, 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,
         )
         self.log.info("Starting Data Fusion pipeline: %s", self.pipeline_name)
         instance = hook.get_instance(
@@ -725,12 +842,22 @@ class CloudDataFusionStopPipelineOperator(BaseOperator):
     :type api_version: str
     :param gcp_conn_id: The connection ID to use when fetching connection info.
     :type gcp_conn_id: str
-    :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 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.
     :type delegate_to: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ("instance_name", "pipeline_name")
+    template_fields = ("instance_name", "pipeline_name", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -743,6 +870,7 @@ class CloudDataFusionStopPipelineOperator(BaseOperator):
         api_version: str = "v1beta1",
         gcp_conn_id: str = "google_cloud_default",
         delegate_to: Optional[str] = None,
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -754,12 +882,14 @@ class CloudDataFusionStopPipelineOperator(BaseOperator):
         self.api_version = api_version
         self.gcp_conn_id = gcp_conn_id
         self.delegate_to = delegate_to
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, 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,
         )
         self.log.info("Starting Data Fusion pipeline: %s", self.pipeline_name)
         instance = hook.get_instance(
diff --git a/airflow/providers/google/cloud/operators/dataproc.py b/airflow/providers/google/cloud/operators/dataproc.py
index 113fe63..2855188 100644
--- a/airflow/providers/google/cloud/operators/dataproc.py
+++ b/airflow/providers/google/cloud/operators/dataproc.py
@@ -133,10 +133,6 @@ class ClusterGenerator:
     :type region: str
     :param gcp_conn_id: The connection ID to use connecting to Google Cloud Platform.
     :type gcp_conn_id: str
-    :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.
-    :type delegate_to: str
     :param service_account: The service account of the dataproc instances.
     :type service_account: str
     :param service_account_scopes: The URIs of service account scopes to be included.
@@ -460,9 +456,18 @@ class DataprocCreateClusterOperator(BaseOperator):
     :type metadata: Sequence[Tuple[str, str]]
     :param gcp_conn_id: The connection ID to use connecting to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ('project_id', 'region', 'cluster')
+    template_fields = ('project_id', 'region', 'cluster', 'impersonation_chain',)
 
     @apply_defaults
     def __init__(  # pylint: disable=too-many-arguments
@@ -478,6 +483,7 @@ class DataprocCreateClusterOperator(BaseOperator):
         timeout: float = 1 * 60 * 60,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         # TODO: remove one day
@@ -520,6 +526,7 @@ class DataprocCreateClusterOperator(BaseOperator):
         self.gcp_conn_id = gcp_conn_id
         self.delete_on_error = delete_on_error
         self.use_if_exists = use_if_exists
+        self.impersonation_chain = impersonation_chain
 
     def _create_cluster(self, hook):
         operation = hook.create_cluster(
@@ -602,7 +609,10 @@ class DataprocCreateClusterOperator(BaseOperator):
 
     def execute(self, context):
         self.log.info('Creating cluster: %s', self.cluster_name)
-        hook = DataprocHook(gcp_conn_id=self.gcp_conn_id)
+        hook = DataprocHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         try:
             # First try to create a new cluster
             cluster = self._create_cluster(hook)
@@ -664,9 +674,18 @@ class DataprocScaleClusterOperator(BaseOperator):
     :type graceful_decommission_timeout: str
     :param gcp_conn_id: The connection ID to use connecting to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ['cluster_name', 'project_id', 'region']
+    template_fields = ['cluster_name', 'project_id', 'region', 'impersonation_chain', ]
 
     @apply_defaults
     def __init__(self, *,
@@ -677,6 +696,7 @@ class DataprocScaleClusterOperator(BaseOperator):
                  num_preemptible_workers: int = 0,
                  graceful_decommission_timeout: Optional[str] = None,
                  gcp_conn_id: str = "google_cloud_default",
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         super().__init__(**kwargs)
         self.project_id = project_id
@@ -686,6 +706,7 @@ class DataprocScaleClusterOperator(BaseOperator):
         self.num_preemptible_workers = num_preemptible_workers
         self.graceful_decommission_timeout = graceful_decommission_timeout
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
         # TODO: Remove one day
         warnings.warn(
@@ -750,7 +771,10 @@ class DataprocScaleClusterOperator(BaseOperator):
             "config.secondary_worker_config.num_instances"
         ]
 
-        hook = DataprocHook(gcp_conn_id=self.gcp_conn_id)
+        hook = DataprocHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         operation = hook.update_cluster(
             project_id=self.project_id,
             location=self.region,
@@ -790,8 +814,19 @@ class DataprocDeleteClusterOperator(BaseOperator):
     :type metadata: Sequence[Tuple[str, str]]
     :param gcp_conn_id: The connection ID to use connecting to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
+    template_fields = ('impersonation_chain',)
+
     @apply_defaults
     def __init__(
         self, *,
@@ -804,6 +839,7 @@ class DataprocDeleteClusterOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ):
         super().__init__(**kwargs)
@@ -816,9 +852,13 @@ class DataprocDeleteClusterOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context: Dict):
-        hook = DataprocHook(gcp_conn_id=self.gcp_conn_id)
+        hook = DataprocHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         self.log.info("Deleting cluster: %s", self.cluster_name)
         operation = hook.delete_cluster(
             project_id=self.project_id,
@@ -852,9 +892,9 @@ class DataprocJobBaseOperator(BaseOperator):
     :type dataproc_jars: list
     :param gcp_conn_id: The connection ID to use connecting to Google Cloud Platform.
     :type gcp_conn_id: str
-    :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 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.
     :type delegate_to: str
     :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
@@ -869,6 +909,16 @@ class DataprocJobBaseOperator(BaseOperator):
         ``'ERROR'`` and ``'CANCELLED'``, but could change in the future. Defaults to
         ``{'ERROR'}``.
     :type job_error_states: set
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
+
     :var dataproc_job_id: The actual "jobId" as submitted to the Dataproc API.
         This is useful for identifying or linking to the job in the Google Cloud Console
         Dataproc UI, as the actual "jobId" submitted to the Dataproc API is appended with
@@ -888,6 +938,7 @@ class DataprocJobBaseOperator(BaseOperator):
                  labels: Optional[Dict] = None,
                  region: str = 'global',
                  job_error_states: Optional[Set[str]] = None,
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         super().__init__(**kwargs)
         self.gcp_conn_id = gcp_conn_id
@@ -899,8 +950,9 @@ class DataprocJobBaseOperator(BaseOperator):
         self.dataproc_jars = dataproc_jars
         self.region = region
         self.job_error_states = job_error_states if job_error_states is not None else {'ERROR'}
+        self.impersonation_chain = impersonation_chain
 
-        self.hook = DataprocHook(gcp_conn_id=gcp_conn_id)
+        self.hook = DataprocHook(gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain)
         self.project_id = self.hook.project_id
         self.job_template = None
         self.job = None
@@ -1003,7 +1055,7 @@ class DataprocSubmitPigJobOperator(DataprocJobBaseOperator):
     :type variables: dict
     """
     template_fields = ['query', 'variables', 'job_name', 'cluster_name',
-                       'region', 'dataproc_jars', 'dataproc_properties']
+                       'region', 'dataproc_jars', 'dataproc_properties', 'impersonation_chain', ]
     template_ext = ('.pg', '.pig',)
     ui_color = '#0273d4'
     job_type = 'pig_job'
@@ -1068,7 +1120,7 @@ class DataprocSubmitHiveJobOperator(DataprocJobBaseOperator):
     :type variables: dict
     """
     template_fields = ['query', 'variables', 'job_name', 'cluster_name',
-                       'region', 'dataproc_jars', 'dataproc_properties']
+                       'region', 'dataproc_jars', 'dataproc_properties', 'impersonation_chain', ]
     template_ext = ('.q', '.hql',)
     ui_color = '#0273d4'
     job_type = 'hive_job'
@@ -1133,7 +1185,7 @@ class DataprocSubmitSparkSqlJobOperator(DataprocJobBaseOperator):
     :type variables: dict
     """
     template_fields = ['query', 'variables', 'job_name', 'cluster_name',
-                       'region', 'dataproc_jars', 'dataproc_properties']
+                       'region', 'dataproc_jars', 'dataproc_properties', 'impersonation_chain', ]
     template_ext = ('.q',)
     ui_color = '#0273d4'
     job_type = 'spark_sql_job'
@@ -1206,7 +1258,7 @@ class DataprocSubmitSparkJobOperator(DataprocJobBaseOperator):
     """
 
     template_fields = ['arguments', 'job_name', 'cluster_name',
-                       'region', 'dataproc_jars', 'dataproc_properties']
+                       'region', 'dataproc_jars', 'dataproc_properties', 'impersonation_chain', ]
     ui_color = '#0273d4'
     job_type = 'spark_job'
 
@@ -1278,7 +1330,7 @@ class DataprocSubmitHadoopJobOperator(DataprocJobBaseOperator):
     """
 
     template_fields = ['arguments', 'job_name', 'cluster_name',
-                       'region', 'dataproc_jars', 'dataproc_properties']
+                       'region', 'dataproc_jars', 'dataproc_properties', 'impersonation_chain', ]
     ui_color = '#0273d4'
     job_type = 'hadoop_job'
 
@@ -1350,7 +1402,7 @@ class DataprocSubmitPySparkJobOperator(DataprocJobBaseOperator):
     """
 
     template_fields = ['main', 'arguments', 'job_name', 'cluster_name',
-                       'region', 'dataproc_jars', 'dataproc_properties']
+                       'region', 'dataproc_jars', 'dataproc_properties', 'impersonation_chain', ]
     ui_color = '#0273d4'
     job_type = 'pyspark_job'
 
@@ -1372,7 +1424,8 @@ class DataprocSubmitPySparkJobOperator(DataprocJobBaseOperator):
         self.log.info("Uploading %s to %s", local_file, temp_filename)
 
         GCSHook(
-            google_cloud_storage_conn_id=self.gcp_conn_id
+            google_cloud_storage_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain
         ).upload(
             bucket_name=bucket,
             object_name=temp_filename,
@@ -1467,10 +1520,6 @@ class DataprocInstantiateWorkflowTemplateOperator(BaseOperator):
     :type project_id: str
     :param region: leave as 'global', might become relevant in the future
     :type region: str
-    :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.
-    :type delegate_to: str
     :param parameters: a map of parameters for Dataproc Template in key-value format:
         map (key: string, value: string)
         Example: { "date_from": "2019-08-01", "date_to": "2019-08-02"}.
@@ -1495,9 +1544,18 @@ class DataprocInstantiateWorkflowTemplateOperator(BaseOperator):
     :type metadata: Sequence[Tuple[str, str]]
     :param gcp_conn_id: The connection ID to use connecting to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ['template_id']
+    template_fields = ['template_id', 'impersonation_chain', ]
 
     @apply_defaults
     def __init__(  # pylint: disable=too-many-arguments
@@ -1512,6 +1570,7 @@ class DataprocInstantiateWorkflowTemplateOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -1526,9 +1585,13 @@ class DataprocInstantiateWorkflowTemplateOperator(BaseOperator):
         self.metadata = metadata
         self.request_id = request_id
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = DataprocHook(gcp_conn_id=self.gcp_conn_id)
+        hook = DataprocHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         self.log.info('Instantiating template %s', self.template_id)
         operation = hook.instantiate_workflow_template(
             project_id=self.project_id,
@@ -1585,9 +1648,18 @@ class DataprocInstantiateInlineWorkflowTemplateOperator(BaseOperator):
     :type metadata: Sequence[Tuple[str, str]]
     :param gcp_conn_id: The connection ID to use connecting to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ['template']
+    template_fields = ['template', 'impersonation_chain', ]
 
     @apply_defaults
     def __init__(
@@ -1600,6 +1672,7 @@ class DataprocInstantiateInlineWorkflowTemplateOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -1612,10 +1685,14 @@ class DataprocInstantiateInlineWorkflowTemplateOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
         self.log.info('Instantiating Inline Template')
-        hook = DataprocHook(gcp_conn_id=self.gcp_conn_id)
+        hook = DataprocHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         operation = hook.instantiate_inline_workflow_template(
             template=self.template,
             project_id=self.project_id,
@@ -1656,9 +1733,18 @@ class DataprocSubmitJobOperator(BaseOperator):
     :type metadata: Sequence[Tuple[str, str]]
     :param gcp_conn_id:
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ('project_id', 'location', 'job')
+    template_fields = ('project_id', 'location', 'job', 'impersonation_chain',)
 
     @apply_defaults
     def __init__(
@@ -1671,6 +1757,7 @@ class DataprocSubmitJobOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -1682,10 +1769,14 @@ class DataprocSubmitJobOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context: Dict):
         self.log.info("Submitting job")
-        hook = DataprocHook(gcp_conn_id=self.gcp_conn_id)
+        hook = DataprocHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         job_object = hook.submit_job(
             project_id=self.project_id,
             location=self.location,
@@ -1746,7 +1837,17 @@ class DataprocUpdateClusterOperator(BaseOperator):
     :type metadata: Sequence[Tuple[str, str]]
     :param gcp_conn_id: The connection ID to use connecting to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
+    template_fields = ('impersonation_chain',)
 
     @apply_defaults
     def __init__(  # pylint: disable=too-many-arguments
@@ -1762,6 +1863,7 @@ class DataprocUpdateClusterOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ):
         super().__init__(**kwargs)
@@ -1776,9 +1878,13 @@ class DataprocUpdateClusterOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context: Dict):
-        hook = DataprocHook(gcp_conn_id=self.gcp_conn_id)
+        hook = DataprocHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         self.log.info("Updating %s cluster.", self.cluster_name)
         operation = hook.update_cluster(
             project_id=self.project_id,
diff --git a/airflow/providers/google/cloud/operators/datastore.py b/airflow/providers/google/cloud/operators/datastore.py
index c0c9508..15d804d 100644
--- a/airflow/providers/google/cloud/operators/datastore.py
+++ b/airflow/providers/google/cloud/operators/datastore.py
@@ -19,7 +19,7 @@
 """
 This module contains Google Datastore operators.
 """
-from typing import Any, Dict, List, Optional
+from typing import Any, Dict, List, Optional, Sequence, Union
 
 from airflow.exceptions import AirflowException
 from airflow.models import BaseOperator
@@ -46,9 +46,9 @@ class CloudDatastoreExportEntitiesOperator(BaseOperator):
     :param cloud_storage_conn_id: the name of the cloud storage connection id to
         force-write backup
     :type cloud_storage_conn_id: str
-    :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 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.
     :type delegate_to: str
     :param entity_filter: description of what data from the project is included in the
         export, refer to
@@ -62,8 +62,17 @@ class CloudDatastoreExportEntitiesOperator(BaseOperator):
     :param overwrite_existing: if the storage bucket + namespace is not empty, it will be
         emptied prior to exports. This enables overwriting existing backups.
     :type overwrite_existing: bool
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
-    template_fields = ['bucket', 'namespace', 'entity_filter', 'labels']
+    template_fields = ['bucket', 'namespace', 'entity_filter', 'labels', 'impersonation_chain', ]
 
     @apply_defaults
     def __init__(self,  # pylint: disable=too-many-arguments
@@ -78,6 +87,7 @@ class CloudDatastoreExportEntitiesOperator(BaseOperator):
                  polling_interval_in_seconds: int = 10,
                  overwrite_existing: bool = False,
                  project_id: Optional[str] = None,
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         super().__init__(**kwargs)
         self.datastore_conn_id = datastore_conn_id
@@ -90,6 +100,7 @@ class CloudDatastoreExportEntitiesOperator(BaseOperator):
         self.polling_interval_in_seconds = polling_interval_in_seconds
         self.overwrite_existing = overwrite_existing
         self.project_id = project_id
+        self.impersonation_chain = impersonation_chain
         if kwargs.get('xcom_push') is not None:
             raise AirflowException("'xcom_push' was deprecated, use 'BaseOperator.do_xcom_push' instead")
 
@@ -97,12 +108,19 @@ class CloudDatastoreExportEntitiesOperator(BaseOperator):
         self.log.info('Exporting data to Cloud Storage bucket %s', self.bucket)
 
         if self.overwrite_existing and self.namespace:
-            gcs_hook = GCSHook(self.cloud_storage_conn_id)
+            gcs_hook = GCSHook(
+                self.cloud_storage_conn_id,
+                impersonation_chain=self.impersonation_chain
+            )
             objects = gcs_hook.list(self.bucket, prefix=self.namespace)
             for obj in objects:
                 gcs_hook.delete(self.bucket, obj)
 
-        ds_hook = DatastoreHook(self.datastore_conn_id, self.delegate_to)
+        ds_hook = DatastoreHook(
+            self.datastore_conn_id,
+            self.delegate_to,
+            impersonation_chain=self.impersonation_chain,
+        )
         result = ds_hook.export_to_storage_bucket(bucket=self.bucket,
                                                   namespace=self.namespace,
                                                   entity_filter=self.entity_filter,
@@ -143,16 +161,26 @@ class CloudDatastoreImportEntitiesOperator(BaseOperator):
     :type labels: dict
     :param datastore_conn_id: the name of the connection id to use
     :type datastore_conn_id: str
-    :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 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.
     :type delegate_to: str
     :param polling_interval_in_seconds: number of seconds to wait before polling for
         execution status again
     :type polling_interval_in_seconds: float
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ['bucket', 'file', 'namespace', 'entity_filter', 'labels']
+    template_fields = ['bucket', 'file', 'namespace', 'entity_filter', 'labels',
+                       'impersonation_chain', ]
 
     @apply_defaults
     def __init__(self,
@@ -166,6 +194,7 @@ class CloudDatastoreImportEntitiesOperator(BaseOperator):
                  delegate_to: Optional[str] = None,
                  polling_interval_in_seconds: float = 10,
                  project_id: Optional[str] = None,
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         super().__init__(**kwargs)
         self.datastore_conn_id = datastore_conn_id
@@ -177,12 +206,17 @@ class CloudDatastoreImportEntitiesOperator(BaseOperator):
         self.labels = labels
         self.polling_interval_in_seconds = polling_interval_in_seconds
         self.project_id = project_id
+        self.impersonation_chain = impersonation_chain
         if kwargs.get('xcom_push') is not None:
             raise AirflowException("'xcom_push' was deprecated, use 'BaseOperator.do_xcom_push' instead")
 
     def execute(self, context):
         self.log.info('Importing data from Cloud Storage bucket %s', self.bucket)
-        ds_hook = DatastoreHook(self.datastore_conn_id, self.delegate_to)
+        ds_hook = DatastoreHook(
+            self.datastore_conn_id,
+            self.delegate_to,
+            impersonation_chain=self.impersonation_chain,
+        )
         result = ds_hook.import_from_storage_bucket(bucket=self.bucket,
                                                     file=self.file,
                                                     namespace=self.namespace,
@@ -216,14 +250,23 @@ class CloudDatastoreAllocateIdsOperator(BaseOperator):
     :type partial_keys: list
     :param project_id: Google Cloud Platform project ID against which to make the request.
     :type project_id: str
-    :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 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.
     :type delegate_to: str
     :param gcp_conn_id: The connection ID to use connecting to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
-    template_fields = ("partial_keys",)
+    template_fields = ("partial_keys", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -232,6 +275,7 @@ class CloudDatastoreAllocateIdsOperator(BaseOperator):
         project_id: Optional[str] = None,
         delegate_to: Optional[str] = None,
         gcp_conn_id: str = 'google_cloud_default',
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -240,9 +284,13 @@ class CloudDatastoreAllocateIdsOperator(BaseOperator):
         self.gcp_conn_id = gcp_conn_id
         self.project_id = project_id
         self.delegate_to = delegate_to
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = DatastoreHook(gcp_conn_id=self.gcp_conn_id)
+        hook = DatastoreHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         keys = hook.allocate_ids(
             partial_keys=self.partial_keys,
             project_id=self.project_id,
@@ -265,14 +313,23 @@ class CloudDatastoreBeginTransactionOperator(BaseOperator):
     :type transaction_options: Dict[str, Any]
     :param project_id: Google Cloud Platform project ID against which to make the request.
     :type project_id: str
-    :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 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.
     :type delegate_to: str
     :param gcp_conn_id: The connection ID to use connecting to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
-    template_fields = ("transaction_options",)
+    template_fields = ("transaction_options", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -281,6 +338,7 @@ class CloudDatastoreBeginTransactionOperator(BaseOperator):
         project_id: Optional[str] = None,
         delegate_to: Optional[str] = None,
         gcp_conn_id: str = 'google_cloud_default',
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -289,9 +347,13 @@ class CloudDatastoreBeginTransactionOperator(BaseOperator):
         self.gcp_conn_id = gcp_conn_id
         self.project_id = project_id
         self.delegate_to = delegate_to
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = DatastoreHook(gcp_conn_id=self.gcp_conn_id)
+        hook = DatastoreHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         handle = hook.begin_transaction(
             transaction_options=self.transaction_options,
             project_id=self.project_id,
@@ -314,14 +376,23 @@ class CloudDatastoreCommitOperator(BaseOperator):
     :type body: dict
     :param project_id: Google Cloud Platform project ID against which to make the request.
     :type project_id: str
-    :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 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.
     :type delegate_to: str
     :param gcp_conn_id: The connection ID to use connecting to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
-    template_fields = ("body",)
+    template_fields = ("body", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -330,6 +401,7 @@ class CloudDatastoreCommitOperator(BaseOperator):
         project_id: Optional[str] = None,
         delegate_to: Optional[str] = None,
         gcp_conn_id: str = 'google_cloud_default',
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -338,9 +410,13 @@ class CloudDatastoreCommitOperator(BaseOperator):
         self.gcp_conn_id = gcp_conn_id
         self.project_id = project_id
         self.delegate_to = delegate_to
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = DatastoreHook(gcp_conn_id=self.gcp_conn_id)
+        hook = DatastoreHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         response = hook.commit(
             body=self.body,
             project_id=self.project_id,
@@ -363,14 +439,23 @@ class CloudDatastoreRollbackOperator(BaseOperator):
     :type transaction: str
     :param project_id: Google Cloud Platform project ID against which to make the request.
     :type project_id: str
-    :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 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.
     :type delegate_to: str
     :param gcp_conn_id: The connection ID to use connecting to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
-    template_fields = ("transaction",)
+    template_fields = ("transaction", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -379,6 +464,7 @@ class CloudDatastoreRollbackOperator(BaseOperator):
         project_id: Optional[str] = None,
         delegate_to: Optional[str] = None,
         gcp_conn_id: str = 'google_cloud_default',
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -387,9 +473,13 @@ class CloudDatastoreRollbackOperator(BaseOperator):
         self.gcp_conn_id = gcp_conn_id
         self.project_id = project_id
         self.delegate_to = delegate_to
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = DatastoreHook(gcp_conn_id=self.gcp_conn_id)
+        hook = DatastoreHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         hook.rollback(
             transaction=self.transaction,
             project_id=self.project_id,
@@ -411,14 +501,23 @@ class CloudDatastoreRunQueryOperator(BaseOperator):
     :type body: dict
     :param project_id: Google Cloud Platform project ID against which to make the request.
     :type project_id: str
-    :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 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.
     :type delegate_to: str
     :param gcp_conn_id: The connection ID to use connecting to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
-    template_fields = ("body",)
+    template_fields = ("body", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -427,6 +526,7 @@ class CloudDatastoreRunQueryOperator(BaseOperator):
         project_id: Optional[str] = None,
         delegate_to: Optional[str] = None,
         gcp_conn_id: str = 'google_cloud_default',
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -435,9 +535,13 @@ class CloudDatastoreRunQueryOperator(BaseOperator):
         self.gcp_conn_id = gcp_conn_id
         self.project_id = project_id
         self.delegate_to = delegate_to
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = DatastoreHook(gcp_conn_id=self.gcp_conn_id)
+        hook = DatastoreHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         response = hook.run_query(
             body=self.body,
             project_id=self.project_id,
@@ -454,14 +558,23 @@ class CloudDatastoreGetOperationOperator(BaseOperator):
 
     :param name: the name of the operation resource.
     :type name: str
-    :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 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.
     :type delegate_to: str
     :param gcp_conn_id: The connection ID to use connecting to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
-    template_fields = ("name",)
+    template_fields = ("name", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -469,6 +582,7 @@ class CloudDatastoreGetOperationOperator(BaseOperator):
         name: str,
         delegate_to: Optional[str] = None,
         gcp_conn_id: str = 'google_cloud_default',
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -476,9 +590,13 @@ class CloudDatastoreGetOperationOperator(BaseOperator):
         self.name = name
         self.gcp_conn_id = gcp_conn_id
         self.delegate_to = delegate_to
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = DatastoreHook(gcp_conn_id=self.gcp_conn_id)
+        hook = DatastoreHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         op = hook.get_operation(name=self.name)
         return op
 
@@ -492,14 +610,23 @@ class CloudDatastoreDeleteOperationOperator(BaseOperator):
 
     :param name: the name of the operation resource.
     :type name: str
-    :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 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.
     :type delegate_to: str
     :param gcp_conn_id: The connection ID to use connecting to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
-    template_fields = ("name",)
+    template_fields = ("name", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -507,6 +634,7 @@ class CloudDatastoreDeleteOperationOperator(BaseOperator):
         name: str,
         delegate_to: Optional[str] = None,
         gcp_conn_id: str = 'google_cloud_default',
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -514,7 +642,11 @@ class CloudDatastoreDeleteOperationOperator(BaseOperator):
         self.name = name
         self.gcp_conn_id = gcp_conn_id
         self.delegate_to = delegate_to
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = DatastoreHook(gcp_conn_id=self.gcp_conn_id)
+        hook = DatastoreHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         hook.delete_operation(name=self.name)
diff --git a/airflow/providers/google/cloud/operators/dlp.py b/airflow/providers/google/cloud/operators/dlp.py
index 7df731e..43c698e 100644
--- a/airflow/providers/google/cloud/operators/dlp.py
+++ b/airflow/providers/google/cloud/operators/dlp.py
@@ -59,9 +59,18 @@ class CloudDLPCancelDLPJobOperator(BaseOperator):
     :type metadata: sequence[tuple[str, str]]]
     :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ("dlp_job_id", "project_id", "gcp_conn_id")
+    template_fields = ("dlp_job_id", "project_id", "gcp_conn_id", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -72,6 +81,7 @@ class CloudDLPCancelDLPJobOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -81,9 +91,13 @@ class CloudDLPCancelDLPJobOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDLPHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         hook.cancel_dlp_job(
             dlp_job_id=self.dlp_job_id,
             project_id=self.project_id,
@@ -120,6 +134,16 @@ class CloudDLPCreateDeidentifyTemplateOperator(BaseOperator):
     :type metadata: sequence[tuple[str, str]]]
     :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
+
     :rtype: google.cloud.dlp_v2.types.DeidentifyTemplate
     """
 
@@ -129,6 +153,7 @@ class CloudDLPCreateDeidentifyTemplateOperator(BaseOperator):
         "deidentify_template",
         "template_id",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -142,6 +167,7 @@ class CloudDLPCreateDeidentifyTemplateOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -153,9 +179,13 @@ class CloudDLPCreateDeidentifyTemplateOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDLPHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         try:
             template = hook.create_deidentify_template(
                 organization_id=self.organization_id,
@@ -207,10 +237,21 @@ class CloudDLPCreateDLPJobOperator(BaseOperator):
     :type wait_until_finished: bool
     :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
+
     :rtype: google.cloud.dlp_v2.types.DlpJob
     """
 
-    template_fields = ("project_id", "inspect_job", "risk_job", "job_id", "gcp_conn_id")
+    template_fields = ("project_id", "inspect_job", "risk_job", "job_id", "gcp_conn_id",
+                       "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -224,6 +265,7 @@ class CloudDLPCreateDLPJobOperator(BaseOperator):
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         wait_until_finished: bool = True,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -236,9 +278,13 @@ class CloudDLPCreateDLPJobOperator(BaseOperator):
         self.metadata = metadata
         self.wait_until_finished = wait_until_finished
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDLPHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         try:
             job = hook.create_dlp_job(
                 project_id=self.project_id,
@@ -288,6 +334,16 @@ class CloudDLPCreateInspectTemplateOperator(BaseOperator):
     :type metadata: sequence[tuple[str, str]]]
     :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
+
     :rtype: google.cloud.dlp_v2.types.InspectTemplate
     """
 
@@ -297,6 +353,7 @@ class CloudDLPCreateInspectTemplateOperator(BaseOperator):
         "inspect_template",
         "template_id",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -310,6 +367,7 @@ class CloudDLPCreateInspectTemplateOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -321,9 +379,13 @@ class CloudDLPCreateInspectTemplateOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDLPHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         try:
             template = hook.create_inspect_template(
                 organization_id=self.organization_id,
@@ -370,10 +432,21 @@ class CloudDLPCreateJobTriggerOperator(BaseOperator):
     :type metadata: sequence[tuple[str, str]]]
     :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
+
     :rtype: google.cloud.dlp_v2.types.JobTrigger
     """
 
-    template_fields = ("project_id", "job_trigger", "trigger_id", "gcp_conn_id")
+    template_fields = ("project_id", "job_trigger", "trigger_id", "gcp_conn_id",
+                       "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -385,6 +458,7 @@ class CloudDLPCreateJobTriggerOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -395,9 +469,13 @@ class CloudDLPCreateJobTriggerOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDLPHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         try:
             trigger = hook.create_job_trigger(
                 project_id=self.project_id,
@@ -444,6 +522,16 @@ class CloudDLPCreateStoredInfoTypeOperator(BaseOperator):
     :type metadata: sequence[tuple[str, str]]]
     :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
+
     :rtype: google.cloud.dlp_v2.types.StoredInfoType
     """
 
@@ -453,6 +541,7 @@ class CloudDLPCreateStoredInfoTypeOperator(BaseOperator):
         "config",
         "stored_info_type_id",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -466,6 +555,7 @@ class CloudDLPCreateStoredInfoTypeOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -477,9 +567,13 @@ class CloudDLPCreateStoredInfoTypeOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDLPHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         try:
             info = hook.create_stored_info_type(
                 organization_id=self.organization_id,
@@ -538,6 +632,16 @@ class CloudDLPDeidentifyContentOperator(BaseOperator):
     :type metadata: sequence[tuple[str, str]]]
     :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
+
     :rtype: google.cloud.dlp_v2.types.DeidentifyContentResponse
     """
 
@@ -549,6 +653,7 @@ class CloudDLPDeidentifyContentOperator(BaseOperator):
         "inspect_template_name",
         "deidentify_template_name",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -564,6 +669,7 @@ class CloudDLPDeidentifyContentOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -577,9 +683,13 @@ class CloudDLPDeidentifyContentOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDLPHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         response = hook.deidentify_content(
             project_id=self.project_id,
             deidentify_config=self.deidentify_config,
@@ -618,9 +728,19 @@ class CloudDLPDeleteDeidentifyTemplateOperator(BaseOperator):
     :type metadata: sequence[tuple[str, str]]]
     :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ("template_id", "organization_id", "project_id", "gcp_conn_id")
+    template_fields = ("template_id", "organization_id", "project_id", "gcp_conn_id",
+                       "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -632,6 +752,7 @@ class CloudDLPDeleteDeidentifyTemplateOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -642,9 +763,13 @@ class CloudDLPDeleteDeidentifyTemplateOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDLPHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         try:
             hook.delete_deidentify_template(
                 template_id=self.template_id,
@@ -680,9 +805,18 @@ class CloudDLPDeleteDLPJobOperator(BaseOperator):
     :type metadata: sequence[tuple[str, str]]]
     :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ("dlp_job_id", "project_id", "gcp_conn_id")
+    template_fields = ("dlp_job_id", "project_id", "gcp_conn_id", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -693,6 +827,7 @@ class CloudDLPDeleteDLPJobOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -702,9 +837,13 @@ class CloudDLPDeleteDLPJobOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDLPHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         try:
             hook.delete_dlp_job(
                 dlp_job_id=self.dlp_job_id,
@@ -741,9 +880,19 @@ class CloudDLPDeleteInspectTemplateOperator(BaseOperator):
     :type metadata: sequence[tuple[str, str]]]
     :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ("template_id", "organization_id", "project_id", "gcp_conn_id")
+    template_fields = ("template_id", "organization_id", "project_id", "gcp_conn_id",
+                       "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -755,6 +904,7 @@ class CloudDLPDeleteInspectTemplateOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -765,9 +915,13 @@ class CloudDLPDeleteInspectTemplateOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDLPHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         try:
             hook.delete_inspect_template(
                 template_id=self.template_id,
@@ -802,9 +956,18 @@ class CloudDLPDeleteJobTriggerOperator(BaseOperator):
     :type metadata: sequence[tuple[str, str]]]
     :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ("job_trigger_id", "project_id", "gcp_conn_id")
+    template_fields = ("job_trigger_id", "project_id", "gcp_conn_id", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -815,6 +978,7 @@ class CloudDLPDeleteJobTriggerOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -824,9 +988,13 @@ class CloudDLPDeleteJobTriggerOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDLPHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         try:
             hook.delete_job_trigger(
                 job_trigger_id=self.job_trigger_id,
@@ -863,6 +1031,15 @@ class CloudDLPDeleteStoredInfoTypeOperator(BaseOperator):
     :type metadata: sequence[tuple[str, str]]]
     :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = (
@@ -870,6 +1047,7 @@ class CloudDLPDeleteStoredInfoTypeOperator(BaseOperator):
         "organization_id",
         "project_id",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -882,6 +1060,7 @@ class CloudDLPDeleteStoredInfoTypeOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -892,9 +1071,13 @@ class CloudDLPDeleteStoredInfoTypeOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDLPHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         try:
             hook.delete_stored_info_type(
                 stored_info_type_id=self.stored_info_type_id,
@@ -932,10 +1115,21 @@ class CloudDLPGetDeidentifyTemplateOperator(BaseOperator):
     :type metadata: sequence[tuple[str, str]]]
     :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
+
     :rtype: google.cloud.dlp_v2.types.DeidentifyTemplate
     """
 
-    template_fields = ("template_id", "organization_id", "project_id", "gcp_conn_id")
+    template_fields = ("template_id", "organization_id", "project_id", "gcp_conn_id",
+                       "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -947,6 +1141,7 @@ class CloudDLPGetDeidentifyTemplateOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -957,9 +1152,13 @@ class CloudDLPGetDeidentifyTemplateOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDLPHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         template = hook.get_deidentify_template(
             template_id=self.template_id,
             organization_id=self.organization_id,
@@ -992,10 +1191,20 @@ class CloudDLPGetDLPJobOperator(BaseOperator):
     :type metadata: sequence[tuple[str, str]]]
     :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
+
     :rtype: google.cloud.dlp_v2.types.DlpJob
     """
 
-    template_fields = ("dlp_job_id", "project_id", "gcp_conn_id")
+    template_fields = ("dlp_job_id", "project_id", "gcp_conn_id", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -1006,6 +1215,7 @@ class CloudDLPGetDLPJobOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -1015,9 +1225,13 @@ class CloudDLPGetDLPJobOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDLPHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         job = hook.get_dlp_job(
             dlp_job_id=self.dlp_job_id,
             project_id=self.project_id,
@@ -1052,10 +1266,21 @@ class CloudDLPGetInspectTemplateOperator(BaseOperator):
     :type metadata: sequence[tuple[str, str]]]
     :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
+
     :rtype: google.cloud.dlp_v2.types.InspectTemplate
     """
 
-    template_fields = ("template_id", "organization_id", "project_id", "gcp_conn_id")
+    template_fields = ("template_id", "organization_id", "project_id", "gcp_conn_id",
+                       "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -1067,6 +1292,7 @@ class CloudDLPGetInspectTemplateOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -1077,9 +1303,13 @@ class CloudDLPGetInspectTemplateOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDLPHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         template = hook.get_inspect_template(
             template_id=self.template_id,
             organization_id=self.organization_id,
@@ -1112,10 +1342,20 @@ class CloudDLPGetDLPJobTriggerOperator(BaseOperator):
     :type metadata: sequence[tuple[str, str]]]
     :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
+
     :rtype: google.cloud.dlp_v2.types.JobTrigger
     """
 
-    template_fields = ("job_trigger_id", "project_id", "gcp_conn_id")
+    template_fields = ("job_trigger_id", "project_id", "gcp_conn_id", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -1126,6 +1366,7 @@ class CloudDLPGetDLPJobTriggerOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -1135,9 +1376,13 @@ class CloudDLPGetDLPJobTriggerOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDLPHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         trigger = hook.get_job_trigger(
             job_trigger_id=self.job_trigger_id,
             project_id=self.project_id,
@@ -1172,6 +1417,16 @@ class CloudDLPGetStoredInfoTypeOperator(BaseOperator):
     :type metadata: sequence[tuple[str, str]]]
     :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
+
     :rtype: google.cloud.dlp_v2.types.StoredInfoType
     """
 
@@ -1180,6 +1435,7 @@ class CloudDLPGetStoredInfoTypeOperator(BaseOperator):
         "organization_id",
         "project_id",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -1192,6 +1448,7 @@ class CloudDLPGetStoredInfoTypeOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -1202,9 +1459,13 @@ class CloudDLPGetStoredInfoTypeOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDLPHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         info = hook.get_stored_info_type(
             stored_info_type_id=self.stored_info_type_id,
             organization_id=self.organization_id,
@@ -1244,6 +1505,16 @@ class CloudDLPInspectContentOperator(BaseOperator):
     :type metadata: sequence[tuple[str, str]]]
     :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
+
     :rtype: google.cloud.tasks_v2.types.InspectContentResponse
     """
 
@@ -1253,6 +1524,7 @@ class CloudDLPInspectContentOperator(BaseOperator):
         "item",
         "inspect_template_name",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -1266,6 +1538,7 @@ class CloudDLPInspectContentOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -1277,9 +1550,13 @@ class CloudDLPInspectContentOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDLPHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         response = hook.inspect_content(
             project_id=self.project_id,
             inspect_config=self.inspect_config,
@@ -1320,10 +1597,20 @@ class CloudDLPListDeidentifyTemplatesOperator(BaseOperator):
     :type metadata: sequence[tuple[str, str]]]
     :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
+
     :rtype: list[google.cloud.dlp_v2.types.DeidentifyTemplate]
     """
 
-    template_fields = ("organization_id", "project_id", "gcp_conn_id")
+    template_fields = ("organization_id", "project_id", "gcp_conn_id", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -1336,6 +1623,7 @@ class CloudDLPListDeidentifyTemplatesOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -1347,9 +1635,13 @@ class CloudDLPListDeidentifyTemplatesOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDLPHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         template = hook.list_deidentify_templates(
             organization_id=self.organization_id,
             project_id=self.project_id,
@@ -1391,10 +1683,20 @@ class CloudDLPListDLPJobsOperator(BaseOperator):
     :type metadata: sequence[tuple[str, str]]]
     :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
+
     :rtype: list[google.cloud.dlp_v2.types.DlpJob]
     """
 
-    template_fields = ("project_id", "gcp_conn_id")
+    template_fields = ("project_id", "gcp_conn_id", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -1408,6 +1710,7 @@ class CloudDLPListDLPJobsOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -1420,9 +1723,13 @@ class CloudDLPListDLPJobsOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDLPHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         job = hook.list_dlp_jobs(
             project_id=self.project_id,
             results_filter=self.results_filter,
@@ -1457,10 +1764,20 @@ class CloudDLPListInfoTypesOperator(BaseOperator):
     :type metadata: sequence[tuple[str, str]]]
     :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
+
     :rtype: ListInfoTypesResponse
     """
 
-    template_fields = ("language_code", "gcp_conn_id")
+    template_fields = ("language_code", "gcp_conn_id", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -1471,6 +1788,7 @@ class CloudDLPListInfoTypesOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -1480,9 +1798,13 @@ class CloudDLPListInfoTypesOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDLPHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         response = hook.list_info_types(
             language_code=self.language_code,
             results_filter=self.results_filter,
@@ -1521,10 +1843,20 @@ class CloudDLPListInspectTemplatesOperator(BaseOperator):
     :type metadata: sequence[tuple[str, str]]]
     :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
+
     :rtype: list[google.cloud.dlp_v2.types.InspectTemplate]
     """
 
-    template_fields = ("organization_id", "project_id", "gcp_conn_id")
+    template_fields = ("organization_id", "project_id", "gcp_conn_id", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -1537,6 +1869,7 @@ class CloudDLPListInspectTemplatesOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -1548,9 +1881,13 @@ class CloudDLPListInspectTemplatesOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDLPHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         templates = hook.list_inspect_templates(
             organization_id=self.organization_id,
             project_id=self.project_id,
@@ -1590,10 +1927,20 @@ class CloudDLPListJobTriggersOperator(BaseOperator):
     :type metadata: sequence[tuple[str, str]]]
     :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
+
     :rtype: list[google.cloud.dlp_v2.types.JobTrigger]
     """
 
-    template_fields = ("project_id", "gcp_conn_id")
+    template_fields = ("project_id", "gcp_conn_id", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -1606,6 +1953,7 @@ class CloudDLPListJobTriggersOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -1617,9 +1965,13 @@ class CloudDLPListJobTriggersOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDLPHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         jobs = hook.list_job_triggers(
             project_id=self.project_id,
             page_size=self.page_size,
@@ -1660,10 +2012,20 @@ class CloudDLPListStoredInfoTypesOperator(BaseOperator):
     :type metadata: sequence[tuple[str, str]]]
     :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
+
     :rtype: list[google.cloud.dlp_v2.types.StoredInfoType]
     """
 
-    template_fields = ("organization_id", "project_id", "gcp_conn_id")
+    template_fields = ("organization_id", "project_id", "gcp_conn_id", "impersonation_chain",)
 
     @apply_defaults
     def __init__(
@@ -1676,6 +2038,7 @@ class CloudDLPListStoredInfoTypesOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -1687,9 +2050,13 @@ class CloudDLPListStoredInfoTypesOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDLPHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         infos = hook.list_stored_info_types(
             organization_id=self.organization_id,
             project_id=self.project_id,
@@ -1734,6 +2101,16 @@ class CloudDLPRedactImageOperator(BaseOperator):
     :type metadata: sequence[tuple[str, str]]]
     :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
+
     :rtype: google.cloud.dlp_v2.types.RedactImageResponse
     """
 
@@ -1744,6 +2121,7 @@ class CloudDLPRedactImageOperator(BaseOperator):
         "include_findings",
         "byte_item",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -1758,6 +2136,7 @@ class CloudDLPRedactImageOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -1770,9 +2149,13 @@ class CloudDLPRedactImageOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDLPHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         response = hook.redact_image(
             project_id=self.project_id,
             inspect_config=self.inspect_config,
@@ -1819,6 +2202,16 @@ class CloudDLPReidentifyContentOperator(BaseOperator):
     :type metadata: sequence[tuple[str, str]]]
     :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
+
     :rtype: google.cloud.dlp_v2.types.ReidentifyContentResponse
     """
 
@@ -1830,6 +2223,7 @@ class CloudDLPReidentifyContentOperator(BaseOperator):
         "inspect_template_name",
         "reidentify_template_name",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -1845,6 +2239,7 @@ class CloudDLPReidentifyContentOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -1858,9 +2253,13 @@ class CloudDLPReidentifyContentOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDLPHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         response = hook.reidentify_content(
             project_id=self.project_id,
             reidentify_config=self.reidentify_config,
@@ -1903,6 +2302,16 @@ class CloudDLPUpdateDeidentifyTemplateOperator(BaseOperator):
     :type metadata: sequence[tuple[str, str]]]
     :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
+
     :rtype: google.cloud.dlp_v2.types.DeidentifyTemplate
     """
 
@@ -1913,6 +2322,7 @@ class CloudDLPUpdateDeidentifyTemplateOperator(BaseOperator):
         "deidentify_template",
         "update_mask",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -1927,6 +2337,7 @@ class CloudDLPUpdateDeidentifyTemplateOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -1939,9 +2350,13 @@ class CloudDLPUpdateDeidentifyTemplateOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDLPHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         template = hook.update_deidentify_template(
             template_id=self.template_id,
             organization_id=self.organization_id,
@@ -1983,6 +2398,16 @@ class CloudDLPUpdateInspectTemplateOperator(BaseOperator):
     :type metadata: sequence[tuple[str, str]]]
     :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
+
     :rtype: google.cloud.dlp_v2.types.InspectTemplate
     """
 
@@ -1993,6 +2418,7 @@ class CloudDLPUpdateInspectTemplateOperator(BaseOperator):
         "inspect_template",
         "update_mask",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -2007,6 +2433,7 @@ class CloudDLPUpdateInspectTemplateOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -2019,9 +2446,13 @@ class CloudDLPUpdateInspectTemplateOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDLPHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         template = hook.update_inspect_template(
             template_id=self.template_id,
             organization_id=self.organization_id,
@@ -2060,6 +2491,16 @@ class CloudDLPUpdateJobTriggerOperator(BaseOperator):
     :type metadata: sequence[tuple[str, str]]]
     :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
+
     :rtype: google.cloud.dlp_v2.types.InspectTemplate
     """
 
@@ -2069,6 +2510,7 @@ class CloudDLPUpdateJobTriggerOperator(BaseOperator):
         "job_trigger",
         "update_mask",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -2082,6 +2524,7 @@ class CloudDLPUpdateJobTriggerOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -2093,9 +2536,13 @@ class CloudDLPUpdateJobTriggerOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDLPHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         trigger = hook.update_job_trigger(
             job_trigger_id=self.job_trigger_id,
             project_id=self.project_id,
@@ -2137,6 +2584,16 @@ class CloudDLPUpdateStoredInfoTypeOperator(BaseOperator):
     :type metadata: sequence[tuple[str, str]]]
     :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
+
     :rtype: google.cloud.dlp_v2.types.StoredInfoType
     """
 
@@ -2147,6 +2604,7 @@ class CloudDLPUpdateStoredInfoTypeOperator(BaseOperator):
         "config",
         "update_mask",
         "gcp_conn_id",
+        "impersonation_chain",
     )
 
     @apply_defaults
@@ -2161,6 +2619,7 @@ class CloudDLPUpdateStoredInfoTypeOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -2173,9 +2632,13 @@ class CloudDLPUpdateStoredInfoTypeOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudDLPHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         info = hook.update_stored_info_type(
             stored_info_type_id=self.stored_info_type_id,
             organization_id=self.organization_id,
diff --git a/airflow/providers/google/cloud/operators/functions.py b/airflow/providers/google/cloud/operators/functions.py
index 245205a..4a193006 100644
--- a/airflow/providers/google/cloud/operators/functions.py
+++ b/airflow/providers/google/cloud/operators/functions.py
@@ -20,7 +20,7 @@ This module contains Google Cloud Functions operators.
 """
 
 import re
-from typing import Any, Dict, List, Optional
+from typing import Any, Dict, List, Optional, Sequence, Union
 
 from googleapiclient.errors import HttpError
 
@@ -116,9 +116,19 @@ class CloudFunctionDeployFunctionOperator(BaseOperator):
     :type zip_path: str
     :param validate_body: If set to False, body validation is not performed.
     :type validate_body: bool
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
     # [START gcf_function_deploy_template_fields]
-    template_fields = ('body', 'project_id', 'location', 'gcp_conn_id', 'api_version')
+    template_fields = ('body', 'project_id', 'location', 'gcp_conn_id', 'api_version',
+                       'impersonation_chain',)
     # [END gcf_function_deploy_template_fields]
 
     @apply_defaults
@@ -130,6 +140,7 @@ class CloudFunctionDeployFunctionOperator(BaseOperator):
                  api_version: str = 'v1',
                  zip_path: Optional[str] = None,
                  validate_body: bool = True,
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         self.project_id = project_id
         self.location = location
@@ -139,6 +150,7 @@ class CloudFunctionDeployFunctionOperator(BaseOperator):
         self.zip_path = zip_path
         self.zip_path_preprocessor = ZipPathPreprocessor(body, zip_path)
         self._field_validator = None  # type: Optional[GcpBodyFieldValidator]
+        self.impersonation_chain = impersonation_chain
         if validate_body:
             self._field_validator = GcpBodyFieldValidator(CLOUD_FUNCTION_VALIDATION,
                                                           api_version=api_version)
@@ -191,7 +203,11 @@ class CloudFunctionDeployFunctionOperator(BaseOperator):
             {'airflow-version': 'v' + version.replace('.', '-').replace('+', '-')})
 
     def execute(self, context):
-        hook = CloudFunctionsHook(gcp_conn_id=self.gcp_conn_id, api_version=self.api_version)
+        hook = CloudFunctionsHook(
+            gcp_conn_id=self.gcp_conn_id,
+            api_version=self.api_version,
+            impersonation_chain=self.impersonation_chain,
+        )
         if self.zip_path_preprocessor.should_upload_function():
             self.body[GCF_SOURCE_UPLOAD_URL] = self._upload_source_code(hook)
         self._validate_all_body_fields()
@@ -308,9 +324,18 @@ class CloudFunctionDeleteFunctionOperator(BaseOperator):
     :type gcp_conn_id: str
     :param api_version: API version used (for example v1 or v1beta1).
     :type api_version: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
     # [START gcf_function_delete_template_fields]
-    template_fields = ('name', 'gcp_conn_id', 'api_version')
+    template_fields = ('name', 'gcp_conn_id', 'api_version', 'impersonation_chain',)
     # [END gcf_function_delete_template_fields]
 
     @apply_defaults
@@ -318,10 +343,12 @@ class CloudFunctionDeleteFunctionOperator(BaseOperator):
                  name: str,
                  gcp_conn_id: str = 'google_cloud_default',
                  api_version: str = 'v1',
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         self.name = name
         self.gcp_conn_id = gcp_conn_id
         self.api_version = api_version
+        self.impersonation_chain = impersonation_chain
         self._validate_inputs()
         super().__init__(**kwargs)
 
@@ -335,7 +362,11 @@ class CloudFunctionDeleteFunctionOperator(BaseOperator):
                     'Parameter name must match pattern: {}'.format(FUNCTION_NAME_PATTERN))
 
     def execute(self, context):
-        hook = CloudFunctionsHook(gcp_conn_id=self.gcp_conn_id, api_version=self.api_version)
+        hook = CloudFunctionsHook(
+            gcp_conn_id=self.gcp_conn_id,
+            api_version=self.api_version,
+            impersonation_chain=self.impersonation_chain,
+        )
         try:
             return hook.delete_function(self.name)
         except HttpError as e:
@@ -365,9 +396,20 @@ class CloudFunctionInvokeFunctionOperator(BaseOperator):
     :param project_id: Optional, Google Cloud Project project_id where the function belongs.
         If set to None or missing, the default project_id from the GCP connection is used.
     :type project_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
+
     :return: None
     """
-    template_fields = ('function_id', 'input_data', 'location', 'project_id')
+    template_fields = ('function_id', 'input_data', 'location', 'project_id',
+                       'impersonation_chain',)
 
     @apply_defaults
     def __init__(
@@ -378,6 +420,7 @@ class CloudFunctionInvokeFunctionOperator(BaseOperator):
         project_id: Optional[str] = None,
         gcp_conn_id: str = 'google_cloud_default',
         api_version: str = 'v1',
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -387,9 +430,14 @@ class CloudFunctionInvokeFunctionOperator(BaseOperator):
         self.project_id = project_id
         self.gcp_conn_id = gcp_conn_id
         self.api_version = api_version
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context: Dict):
-        hook = CloudFunctionsHook(api_version=self.api_version, gcp_conn_id=self.gcp_conn_id)
+        hook = CloudFunctionsHook(
+            api_version=self.api_version,
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
         self.log.info('Calling function %s.', self.function_id)
         result = hook.call_function(
             function_id=self.function_id,
diff --git a/airflow/providers/google/cloud/operators/gcs.py b/airflow/providers/google/cloud/operators/gcs.py
index ddfb13d..06e03e7 100644
--- a/airflow/providers/google/cloud/operators/gcs.py
+++ b/airflow/providers/google/cloud/operators/gcs.py
@@ -22,7 +22,7 @@ import subprocess
 import sys
 import warnings
 from tempfile import NamedTemporaryFile
-from typing import Dict, Iterable, List, Optional, Union
+from typing import Dict, Iterable, List, Optional, Sequence, Union
 
 from google.api_core.exceptions import Conflict
 
@@ -75,10 +75,19 @@ class GCSCreateBucketOperator(BaseOperator):
     :param google_cloud_storage_conn_id: (Deprecated) The connection ID used to connect to Google Cloud
         Platform. This parameter has been deprecated. You should pass the gcp_conn_id parameter instead.
     :type google_cloud_storage_conn_id: str
-    :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 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.
     :type delegate_to: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
 
     The following Operator would create a new bucket ``test-bucket``
     with ``MULTI_REGIONAL`` storage class in ``EU`` region
@@ -96,7 +105,7 @@ class GCSCreateBucketOperator(BaseOperator):
 
     """
     template_fields = ('bucket_name', 'storage_class',
-                       'location', 'project_id')
+                       'location', 'project_id', 'impersonation_chain',)
     ui_color = '#f0eee4'
 
     @apply_defaults
@@ -110,6 +119,7 @@ class GCSCreateBucketOperator(BaseOperator):
                  gcp_conn_id: str = 'google_cloud_default',
                  google_cloud_storage_conn_id: Optional[str] = None,
                  delegate_to: Optional[str] = None,
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         super().__init__(**kwargs)
 
@@ -127,11 +137,13 @@ class GCSCreateBucketOperator(BaseOperator):
         self.labels = labels
         self.gcp_conn_id = gcp_conn_id
         self.delegate_to = delegate_to
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
         hook = GCSHook(
             google_cloud_storage_conn_id=self.gcp_conn_id,
-            delegate_to=self.delegate_to
+            delegate_to=self.delegate_to,
+            impersonation_chain=self.impersonation_chain,
         )
         try:
             hook.create_bucket(bucket_name=self.bucket_name,
@@ -165,10 +177,19 @@ class GCSListObjectsOperator(BaseOperator):
     :param google_cloud_storage_conn_id: (Deprecated) The connection ID used to connect to Google Cloud
         Platform. This parameter has been deprecated. You should pass the gcp_conn_id parameter instead.
     :type google_cloud_storage_conn_id:
-    :param delegate_to: The account to impersonate, if any.
-        For this to work, the service account making the request must have
+    :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.
     :type delegate_to: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
 
     **Example**:
         The following Operator would list all the Avro files from ``sales/sales-2017``
@@ -182,7 +203,7 @@ class GCSListObjectsOperator(BaseOperator):
                 gcp_conn_id=google_cloud_conn_id
             )
     """
-    template_fields: Iterable[str] = ('bucket', 'prefix', 'delimiter')
+    template_fields: Iterable[str] = ('bucket', 'prefix', 'delimiter', 'impersonation_chain',)
 
     ui_color = '#f0eee4'
 
@@ -194,6 +215,7 @@ class GCSListObjectsOperator(BaseOperator):
                  gcp_conn_id: str = 'google_cloud_default',
                  google_cloud_storage_conn_id: Optional[str] = None,
                  delegate_to: Optional[str] = None,
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         super().__init__(**kwargs)
 
@@ -208,12 +230,14 @@ class GCSListObjectsOperator(BaseOperator):
         self.delimiter = delimiter
         self.gcp_conn_id = gcp_conn_id
         self.delegate_to = delegate_to
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
 
         hook = GCSHook(
             google_cloud_storage_conn_id=self.gcp_conn_id,
-            delegate_to=self.delegate_to
+            delegate_to=self.delegate_to,
+            impersonation_chain=self.impersonation_chain,
         )
 
         self.log.info('Getting list of the files. Bucket: %s; Delimiter: %s; Prefix: %s',
@@ -242,13 +266,22 @@ class GCSDeleteObjectsOperator(BaseOperator):
     :param google_cloud_storage_conn_id: (Deprecated) The connection ID used to connect to Google Cloud
         Platform. This parameter has been deprecated. You should pass the gcp_conn_id parameter instead.
     :type google_cloud_storage_conn_id: str
-    :param delegate_to: The account to impersonate, if any.
-        For this to work, the service account making the request must have
+    :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.
     :type delegate_to: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ('bucket_name', 'prefix', 'objects')
+    template_fields = ('bucket_name', 'prefix', 'objects', 'impersonation_chain',)
 
     @apply_defaults
     def __init__(self, *,
@@ -258,6 +291,7 @@ class GCSDeleteObjectsOperator(BaseOperator):
                  gcp_conn_id: str = 'google_cloud_default',
                  google_cloud_storage_conn_id: Optional[str] = None,
                  delegate_to: Optional[str] = None,
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
 
         if google_cloud_storage_conn_id:
@@ -271,6 +305,7 @@ class GCSDeleteObjectsOperator(BaseOperator):
         self.prefix = prefix
         self.gcp_conn_id = gcp_conn_id
         self.delegate_to = delegate_to
+        self.impersonation_chain = impersonation_chain
 
         if not objects and not prefix:
             raise ValueError("Either object or prefix should be set. Both are None")
@@ -280,7 +315,8 @@ class GCSDeleteObjectsOperator(BaseOperator):
     def execute(self, context):
         hook = GCSHook(
             google_cloud_storage_conn_id=self.gcp_conn_id,
-            delegate_to=self.delegate_to
+            delegate_to=self.delegate_to,
+            impersonation_chain=self.impersonation_chain,
         )
 
         if self.objects:
@@ -321,9 +357,18 @@ class GCSBucketCreateAclEntryOperator(BaseOperator):
     :param google_cloud_storage_conn_id: (Deprecated) The connection ID used to connect to Google Cloud
         Platform. This parameter has been deprecated. You should pass the gcp_conn_id parameter instead.
     :type google_cloud_storage_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
     # [START gcs_bucket_create_acl_template_fields]
-    template_fields = ('bucket', 'entity', 'role', 'user_project')
+    template_fields = ('bucket', 'entity', 'role', 'user_project', 'impersonation_chain',)
     # [END gcs_bucket_create_acl_template_fields]
 
     @apply_defaults
@@ -335,6 +380,7 @@ class GCSBucketCreateAclEntryOperator(BaseOperator):
         user_project: Optional[str] = None,
         gcp_conn_id: str = 'google_cloud_default',
         google_cloud_storage_conn_id: Optional[str] = None,
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -350,10 +396,12 @@ class GCSBucketCreateAclEntryOperator(BaseOperator):
         self.role = role
         self.user_project = user_project
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
         hook = GCSHook(
-            google_cloud_storage_conn_id=self.gcp_conn_id
+            google_cloud_storage_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
         )
         hook.insert_bucket_acl(bucket_name=self.bucket, entity=self.entity, role=self.role,
                                user_project=self.user_project)
@@ -390,9 +438,19 @@ class GCSObjectCreateAclEntryOperator(BaseOperator):
     :param google_cloud_storage_conn_id: (Deprecated) The connection ID used to connect to Google Cloud
         Platform. This parameter has been deprecated. You should pass the gcp_conn_id parameter instead.
     :type google_cloud_storage_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
     # [START gcs_object_create_acl_template_fields]
-    template_fields = ('bucket', 'object_name', 'entity', 'generation', 'role', 'user_project')
+    template_fields = ('bucket', 'object_name', 'entity', 'generation', 'role', 'user_project',
+                       'impersonation_chain',)
     # [END gcs_object_create_acl_template_fields]
 
     @apply_defaults
@@ -405,6 +463,7 @@ class GCSObjectCreateAclEntryOperator(BaseOperator):
                  user_project: Optional[str] = None,
                  gcp_conn_id: str = 'google_cloud_default',
                  google_cloud_storage_conn_id: Optional[str] = None,
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         super().__init__(**kwargs)
 
@@ -421,10 +480,12 @@ class GCSObjectCreateAclEntryOperator(BaseOperator):
         self.generation = generation
         self.user_project = user_project
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
         hook = GCSHook(
-            google_cloud_storage_conn_id=self.gcp_conn_id
+            google_cloud_storage_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
         )
         hook.insert_object_acl(bucket_name=self.bucket,
                                object_name=self.object_name,
@@ -457,9 +518,19 @@ class GCSFileTransformOperator(BaseOperator):
     :type transform_script: Union[str, List[str]]
     :param gcp_conn_id: The connection ID to use connecting to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ('source_bucket', 'destination_bucket', 'transform_script')
+    template_fields = ('source_bucket', 'destination_bucket', 'transform_script',
+                       'impersonation_chain',)
 
     @apply_defaults
     def __init__(
@@ -470,6 +541,7 @@ class GCSFileTransformOperator(BaseOperator):
         destination_bucket: Optional[str] = None,
         destination_object: Optional[str] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -481,9 +553,10 @@ class GCSFileTransformOperator(BaseOperator):
         self.gcp_conn_id = gcp_conn_id
         self.transform_script = transform_script
         self.output_encoding = sys.getdefaultencoding()
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context: Dict):
-        hook = GCSHook(gcp_conn_id=self.gcp_conn_id)
+        hook = GCSHook(gcp_conn_id=self.gcp_conn_id, impersonation_chain=self.impersonation_chain)
 
         with NamedTemporaryFile() as source_file, NamedTemporaryFile() as destination_file:
             self.log.info("Downloading file from %s", self.source_bucket)
@@ -543,24 +616,37 @@ class GCSDeleteBucketOperator(BaseOperator):
     :param force: false not allow to delete non empty bucket, set force=True
         allows to delete non empty bucket
     :type: bool
+    :param gcp_conn_id: The connection ID to use connecting to Google Cloud Platform.
+    :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ('bucket_name', "gcp_conn_id")
+    template_fields = ('bucket_name', "gcp_conn_id", "impersonation_chain",)
 
     @apply_defaults
     def __init__(self, *,
                  bucket_name: str,
                  force: bool = True,
                  gcp_conn_id: str = 'google_cloud_default',
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         super().__init__(**kwargs)
 
         self.bucket_name = bucket_name
         self.force: bool = force
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = GCSHook(gcp_conn_id=self.gcp_conn_id)
+        hook = GCSHook(gcp_conn_id=self.gcp_conn_id, impersonation_chain=self.impersonation_chain)
         hook.delete_bucket(bucket_name=self.bucket_name, force=self.force)
 
 
@@ -599,6 +685,21 @@ class GCSSynchronizeBucketsOperator(BaseOperator):
             This option can delete data quickly if you specify the wrong source/destination combination.
 
     :type delete_extra_files: bool
+    :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform.
+    :type gcp_conn_id: str
+    :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.
+    :type delegate_to: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = (
@@ -611,6 +712,7 @@ class GCSSynchronizeBucketsOperator(BaseOperator):
         'allow_overwrite',
         'gcp_conn_id',
         'delegate_to',
+        'impersonation_chain',
     )
 
     @apply_defaults
@@ -625,6 +727,7 @@ class GCSSynchronizeBucketsOperator(BaseOperator):
         allow_overwrite: bool = False,
         gcp_conn_id: str = 'google_cloud_default',
         delegate_to: Optional[str] = None,
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -637,11 +740,13 @@ class GCSSynchronizeBucketsOperator(BaseOperator):
         self.allow_overwrite = allow_overwrite
         self.gcp_conn_id = gcp_conn_id
         self.delegate_to = delegate_to
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
         hook = GCSHook(
             google_cloud_storage_conn_id=self.gcp_conn_id,
-            delegate_to=self.delegate_to
+            delegate_to=self.delegate_to,
+            impersonation_chain=self.impersonation_chain,
         )
         hook.sync(
             source_bucket=self.source_bucket,
diff --git a/airflow/providers/google/cloud/operators/kubernetes_engine.py b/airflow/providers/google/cloud/operators/kubernetes_engine.py
index a9aed1e..ab671f1 100644
--- a/airflow/providers/google/cloud/operators/kubernetes_engine.py
+++ b/airflow/providers/google/cloud/operators/kubernetes_engine.py
@@ -22,7 +22,7 @@ This module contains Google Kubernetes Engine operators.
 
 import os
 import tempfile
-from typing import Dict, Optional, Union
+from typing import Dict, Optional, Sequence, Union
 
 from google.cloud.container_v1.types import Cluster
 
@@ -69,8 +69,18 @@ class GKEDeleteClusterOperator(BaseOperator):
     :type gcp_conn_id: str
     :param api_version: The api version to use
     :type api_version: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
-    template_fields = ['project_id', 'gcp_conn_id', 'name', 'location', 'api_version']
+    template_fields = ['project_id', 'gcp_conn_id', 'name', 'location', 'api_version',
+                       'impersonation_chain', ]
 
     @apply_defaults
     def __init__(self,
@@ -80,6 +90,7 @@ class GKEDeleteClusterOperator(BaseOperator):
                  project_id: Optional[str] = None,
                  gcp_conn_id: str = 'google_cloud_default',
                  api_version: str = 'v2',
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         super().__init__(**kwargs)
 
@@ -88,6 +99,7 @@ class GKEDeleteClusterOperator(BaseOperator):
         self.location = location
         self.api_version = api_version
         self.name = name
+        self.impersonation_chain = impersonation_chain
         self._check_input()
 
     def _check_input(self):
@@ -97,7 +109,11 @@ class GKEDeleteClusterOperator(BaseOperator):
             raise AirflowException('Operator has incorrect or missing input.')
 
     def execute(self, context):
-        hook = GKEHook(gcp_conn_id=self.gcp_conn_id, location=self.location)
+        hook = GKEHook(
+            gcp_conn_id=self.gcp_conn_id,
+            location=self.location,
+            impersonation_chain=self.impersonation_chain,
+        )
         delete_result = hook.delete_cluster(name=self.name, project_id=self.project_id)
         return delete_result
 
@@ -148,8 +164,18 @@ class GKECreateClusterOperator(BaseOperator):
     :type gcp_conn_id: str
     :param api_version: The api version to use
     :type api_version: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
-    template_fields = ['project_id', 'gcp_conn_id', 'location', 'api_version', 'body']
+    template_fields = ['project_id', 'gcp_conn_id', 'location', 'api_version', 'body',
+                       'impersonation_chain', ]
 
     @apply_defaults
     def __init__(self,
@@ -159,6 +185,7 @@ class GKECreateClusterOperator(BaseOperator):
                  project_id: Optional[str] = None,
                  gcp_conn_id: str = 'google_cloud_default',
                  api_version: str = 'v2',
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         super().__init__(**kwargs)
 
@@ -167,6 +194,7 @@ class GKECreateClusterOperator(BaseOperator):
         self.location = location
         self.api_version = api_version
         self.body = body
+        self.impersonation_chain = impersonation_chain
         self._check_input()
 
     def _check_input(self):
@@ -181,7 +209,11 @@ class GKECreateClusterOperator(BaseOperator):
             raise AirflowException("Operator has incorrect or missing input.")
 
     def execute(self, context):
-        hook = GKEHook(gcp_conn_id=self.gcp_conn_id, location=self.location)
+        hook = GKEHook(
+            gcp_conn_id=self.gcp_conn_id,
+            location=self.location,
+            impersonation_chain=self.impersonation_chain,
+        )
         create_op = hook.create_cluster(cluster=self.body, project_id=self.project_id)
         return create_op
 
diff --git a/airflow/providers/google/cloud/operators/life_sciences.py b/airflow/providers/google/cloud/operators/life_sciences.py
index e176c06..6af8c9d 100644
--- a/airflow/providers/google/cloud/operators/life_sciences.py
+++ b/airflow/providers/google/cloud/operators/life_sciences.py
@@ -17,7 +17,7 @@
 # under the License.
 """Operators that interact with Google Cloud Life Sciences service."""
 
-from typing import Optional
+from typing import Optional, Sequence, Union
 
 from airflow.exceptions import AirflowException
 from airflow.models import BaseOperator
@@ -44,9 +44,18 @@ class LifeSciencesRunPipelineOperator(BaseOperator):
     :type gcp_conn_id: str
     :param api_version: API version used (for example v2beta).
     :type api_version: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
-    template_fields = ("body", "gcp_conn_id", "api_version")
+    template_fields = ("body", "gcp_conn_id", "api_version", "impersonation_chain",)
 
     @apply_defaults
     def __init__(self,
@@ -56,6 +65,7 @@ class LifeSciencesRunPipelineOperator(BaseOperator):
                  project_id: Optional[str] = None,
                  gcp_conn_id: str = "google_cloud_default",
                  api_version: str = "v2beta",
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         super().__init__(**kwargs)
         self.body = body
@@ -64,6 +74,7 @@ class LifeSciencesRunPipelineOperator(BaseOperator):
         self.gcp_conn_id = gcp_conn_id
         self.api_version = api_version
         self._validate_inputs()
+        self.impersonation_chain = impersonation_chain
 
     def _validate_inputs(self):
         if not self.body:
@@ -72,7 +83,11 @@ class LifeSciencesRunPipelineOperator(BaseOperator):
             raise AirflowException("The required parameter 'location' is missing")
 
     def execute(self, context):
-        hook = LifeSciencesHook(gcp_conn_id=self.gcp_conn_id, api_version=self.api_version)
+        hook = LifeSciencesHook(
+            gcp_conn_id=self.gcp_conn_id,
+            api_version=self.api_version,
+            impersonation_chain=self.impersonation_chain,
+        )
 
         return hook.run_pipeline(body=self.body,
                                  location=self.location,
diff --git a/airflow/providers/google/cloud/operators/mlengine.py b/airflow/providers/google/cloud/operators/mlengine.py
index 839b6ad..0b23bc5 100644
--- a/airflow/providers/google/cloud/operators/mlengine.py
+++ b/airflow/providers/google/cloud/operators/mlengine.py
@@ -21,7 +21,7 @@ This module contains GCP MLEngine operators.
 import logging
 import re
 import warnings
-from typing import Dict, List, Optional
+from typing import Dict, List, Optional, Sequence, Union
 
 from airflow.exceptions import AirflowException
 from airflow.models import BaseOperator, BaseOperatorLink
@@ -67,6 +67,7 @@ def _normalize_mlengine_job_id(job_id: str) -> str:
     return cleansed_job_id
 
 
+# pylint: disable=too-many-instance-attributes
 class MLEngineStartBatchPredictionJobOperator(BaseOperator):
     """
     Start a Google Cloud ML Engine prediction job.
@@ -147,12 +148,22 @@ class MLEngineStartBatchPredictionJobOperator(BaseOperator):
     :param gcp_conn_id: The connection ID used for connection to Google
         Cloud Platform.
     :type gcp_conn_id: str
-    :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 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.
     :type delegate_to: str
     :param labels: a dictionary containing labels for the job; passed to BigQuery
     :type labels: Dict[str, str]
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
+
     :raises: ``ValueError``: if a unique model/version origin cannot be
         determined.
     """
@@ -166,6 +177,7 @@ class MLEngineStartBatchPredictionJobOperator(BaseOperator):
         '_model_name',
         '_version_name',
         '_uri',
+        '_impersonation_chain',
     ]
 
     @apply_defaults
@@ -186,6 +198,7 @@ class MLEngineStartBatchPredictionJobOperator(BaseOperator):
                  gcp_conn_id: str = 'google_cloud_default',
                  delegate_to: Optional[str] = None,
                  labels: Optional[Dict[str, str]] = None,
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         super().__init__(**kwargs)
 
@@ -204,6 +217,7 @@ class MLEngineStartBatchPredictionJobOperator(BaseOperator):
         self._gcp_conn_id = gcp_conn_id
         self._delegate_to = delegate_to
         self._labels = labels
+        self._impersonation_chain = impersonation_chain
 
         if not self._project_id:
             raise AirflowException('Google Cloud project id is required.')
@@ -265,7 +279,11 @@ class MLEngineStartBatchPredictionJobOperator(BaseOperator):
             prediction_request['predictionInput'][
                 'signatureName'] = self._signature_name
 
-        hook = MLEngineHook(self._gcp_conn_id, self._delegate_to)
+        hook = MLEngineHook(
+            self._gcp_conn_id,
+            self._delegate_to,
+            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.
@@ -311,15 +329,25 @@ class MLEngineManageModelOperator(BaseOperator):
     :type project_id: str
     :param gcp_conn_id: The connection ID to use when fetching connection info.
     :type gcp_conn_id: str
-    :param delegate_to: The account to impersonate, if any.
-        For this to work, the service account making the request must have
+    :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.
     :type delegate_to: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = [
         '_project_id',
         '_model',
+        '_impersonation_chain',
     ]
 
     @apply_defaults
@@ -330,6 +358,7 @@ class MLEngineManageModelOperator(BaseOperator):
                  project_id: Optional[str] = None,
                  gcp_conn_id: str = 'google_cloud_default',
                  delegate_to: Optional[str] = None,
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         super().__init__(**kwargs)
 
@@ -345,10 +374,14 @@ class MLEngineManageModelOperator(BaseOperator):
         self._operation = operation
         self._gcp_conn_id = gcp_conn_id
         self._delegate_to = delegate_to
+        self._impersonation_chain = impersonation_chain
 
     def execute(self, context):
         hook = MLEngineHook(
-            gcp_conn_id=self._gcp_conn_id, delegate_to=self._delegate_to)
+            gcp_conn_id=self._gcp_conn_id,
+            delegate_to=self._delegate_to,
+            impersonation_chain=self._impersonation_chain,
+        )
         if self._operation == 'create':
             return hook.create_model(project_id=self._project_id, model=self._model)
         elif self._operation == 'get':
@@ -374,15 +407,25 @@ class MLEngineCreateModelOperator(BaseOperator):
     :type project_id: str
     :param gcp_conn_id: The connection ID to use when fetching connection info.
     :type gcp_conn_id: str
-    :param delegate_to: The account to impersonate, if any.
-        For this to work, the service account making the request must have
+    :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.
     :type delegate_to: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = [
         '_project_id',
         '_model',
+        '_impersonation_chain',
     ]
 
     @apply_defaults
@@ -392,16 +435,21 @@ class MLEngineCreateModelOperator(BaseOperator):
                  project_id: Optional[str] = None,
                  gcp_conn_id: str = 'google_cloud_default',
                  delegate_to: Optional[str] = None,
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         super().__init__(**kwargs)
         self._project_id = project_id
         self._model = model
         self._gcp_conn_id = gcp_conn_id
         self._delegate_to = delegate_to
+        self._impersonation_chain = impersonation_chain
 
     def execute(self, context):
         hook = MLEngineHook(
-            gcp_conn_id=self._gcp_conn_id, delegate_to=self._delegate_to)
+            gcp_conn_id=self._gcp_conn_id,
+            delegate_to=self._delegate_to,
+            impersonation_chain=self._impersonation_chain,
+        )
         return hook.create_model(project_id=self._project_id, model=self._model)
 
 
@@ -422,15 +470,25 @@ class MLEngineGetModelOperator(BaseOperator):
     :type project_id: str
     :param gcp_conn_id: The connection ID to use when fetching connection info.
     :type gcp_conn_id: str
-    :param delegate_to: The account to impersonate, if any.
-        For this to work, the service account making the request must have
+    :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.
     :type delegate_to: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = [
         '_project_id',
         '_model_name',
+        '_impersonation_chain',
     ]
 
     @apply_defaults
@@ -440,15 +498,21 @@ class MLEngineGetModelOperator(BaseOperator):
                  project_id: Optional[str] = None,
                  gcp_conn_id: str = 'google_cloud_default',
                  delegate_to: Optional[str] = None,
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         super().__init__(**kwargs)
         self._project_id = project_id
         self._model_name = model_name
         self._gcp_conn_id = gcp_conn_id
         self._delegate_to = delegate_to
+        self._impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = MLEngineHook(gcp_conn_id=self._gcp_conn_id, delegate_to=self._delegate_to)
+        hook = MLEngineHook(
+            gcp_conn_id=self._gcp_conn_id,
+            delegate_to=self._delegate_to,
+            impersonation_chain=self._impersonation_chain,
+        )
         return hook.get_model(project_id=self._project_id, model_name=self._model_name)
 
 
@@ -473,15 +537,25 @@ class MLEngineDeleteModelOperator(BaseOperator):
     :type project_id: str
     :param gcp_conn_id: The connection ID to use when fetching connection info.
     :type gcp_conn_id: str
-    :param delegate_to: The account to impersonate, if any.
-        For this to work, the service account making the request must have
+    :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.
     :type delegate_to: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = [
         '_project_id',
         '_model_name',
+        '_impersonation_chain',
     ]
 
     @apply_defaults
@@ -492,6 +566,7 @@ class MLEngineDeleteModelOperator(BaseOperator):
                  project_id: Optional[str] = None,
                  gcp_conn_id: str = 'google_cloud_default',
                  delegate_to: Optional[str] = None,
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         super().__init__(**kwargs)
         self._project_id = project_id
@@ -499,9 +574,14 @@ class MLEngineDeleteModelOperator(BaseOperator):
         self._delete_contents = delete_contents
         self._gcp_conn_id = gcp_conn_id
         self._delegate_to = delegate_to
+        self._impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = MLEngineHook(gcp_conn_id=self._gcp_conn_id, delegate_to=self._delegate_to)
+        hook = MLEngineHook(
+            gcp_conn_id=self._gcp_conn_id,
+            delegate_to=self._delegate_to,
+            impersonation_chain=self._impersonation_chain
+        )
 
         return hook.delete_model(
             project_id=self._project_id, model_name=self._model_name, delete_contents=self._delete_contents
@@ -556,10 +636,19 @@ class MLEngineManageVersionOperator(BaseOperator):
     :type project_id: str
     :param gcp_conn_id: The connection ID to use when fetching connection info.
     :type gcp_conn_id: str
-    :param delegate_to: The account to impersonate, if any.
-        For this to work, the service account making the request must have
+    :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.
     :type delegate_to: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = [
@@ -567,6 +656,7 @@ class MLEngineManageVersionOperator(BaseOperator):
         '_model_name',
         '_version_name',
         '_version',
+        '_impersonation_chain',
     ]
 
     @apply_defaults
@@ -579,6 +669,7 @@ class MLEngineManageVersionOperator(BaseOperator):
                  project_id: Optional[str] = None,
                  gcp_conn_id: str = 'google_cloud_default',
                  delegate_to: Optional[str] = None,
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         super().__init__(**kwargs)
         self._project_id = project_id
@@ -588,6 +679,7 @@ class MLEngineManageVersionOperator(BaseOperator):
         self._operation = operation
         self._gcp_conn_id = gcp_conn_id
         self._delegate_to = delegate_to
+        self._impersonation_chain = impersonation_chain
 
         warnings.warn(
             "This operator is deprecated. Consider using operators for specific operations: "
@@ -601,7 +693,10 @@ class MLEngineManageVersionOperator(BaseOperator):
             self._version['name'] = self._version_name
 
         hook = MLEngineHook(
-            gcp_conn_id=self._gcp_conn_id, delegate_to=self._delegate_to)
+            gcp_conn_id=self._gcp_conn_id,
+            delegate_to=self._delegate_to,
+            impersonation_chain=self._impersonation_chain,
+        )
 
         if self._operation == 'create':
             if not self._version:
@@ -653,16 +748,26 @@ class MLEngineCreateVersionOperator(BaseOperator):
     :type project_id: str
     :param gcp_conn_id: The connection ID to use when fetching connection info.
     :type gcp_conn_id: str
-    :param delegate_to: The account to impersonate, if any.
-        For this to work, the service account making the request must have
+    :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.
     :type delegate_to: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = [
         '_project_id',
         '_model_name',
         '_version',
+        '_impersonation_chain',
     ]
 
     @apply_defaults
@@ -673,6 +778,7 @@ class MLEngineCreateVersionOperator(BaseOperator):
                  project_id: Optional[str] = None,
                  gcp_conn_id: str = 'google_cloud_default',
                  delegate_to: Optional[str] = None,
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
 
         super().__init__(**kwargs)
@@ -681,6 +787,7 @@ class MLEngineCreateVersionOperator(BaseOperator):
         self._version = version
         self._gcp_conn_id = gcp_conn_id
         self._delegate_to = delegate_to
+        self._impersonation_chain = impersonation_chain
         self._validate_inputs()
 
     def _validate_inputs(self):
@@ -691,7 +798,11 @@ class MLEngineCreateVersionOperator(BaseOperator):
             raise AirflowException("The version parameter could not be empty.")
 
     def execute(self, context):
-        hook = MLEngineHook(gcp_conn_id=self._gcp_conn_id, delegate_to=self._delegate_to)
+        hook = MLEngineHook(
+            gcp_conn_id=self._gcp_conn_id,
+            delegate_to=self._delegate_to,
+            impersonation_chain=self._impersonation_chain,
+        )
 
         return hook.create_version(
             project_id=self._project_id,
@@ -720,16 +831,26 @@ class MLEngineSetDefaultVersionOperator(BaseOperator):
     :type project_id: str
     :param gcp_conn_id: The connection ID to use when fetching connection info.
     :type gcp_conn_id: str
-    :param delegate_to: The account to impersonate, if any.
-        For this to work, the service account making the request must have
+    :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.
     :type delegate_to: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = [
         '_project_id',
         '_model_name',
         '_version_name',
+        '_impersonation_chain',
     ]
 
     @apply_defaults
@@ -740,6 +861,7 @@ class MLEngineSetDefaultVersionOperator(BaseOperator):
                  project_id: Optional[str] = None,
                  gcp_conn_id: str = 'google_cloud_default',
                  delegate_to: Optional[str] = None,
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
 
         super().__init__(**kwargs)
@@ -748,6 +870,7 @@ class MLEngineSetDefaultVersionOperator(BaseOperator):
         self._version_name = version_name
         self._gcp_conn_id = gcp_conn_id
         self._delegate_to = delegate_to
+        self._impersonation_chain = impersonation_chain
         self._validate_inputs()
 
     def _validate_inputs(self):
@@ -758,7 +881,11 @@ class MLEngineSetDefaultVersionOperator(BaseOperator):
             raise AirflowException("The version_name parameter could not be empty.")
 
     def execute(self, context):
-        hook = MLEngineHook(gcp_conn_id=self._gcp_conn_id, delegate_to=self._delegate_to)
+        hook = MLEngineHook(
+            gcp_conn_id=self._gcp_conn_id,
+            delegate_to=self._delegate_to,
+            impersonation_chain=self._impersonation_chain,
+        )
 
         return hook.set_default_version(
             project_id=self._project_id,
@@ -785,14 +912,24 @@ class MLEngineListVersionsOperator(BaseOperator):
     :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 GCP connection is used. (templated)
     :type project_id: str
-    :param delegate_to: The account to impersonate, if any.
-        For this to work, the service account making the request must have
+    :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.
     :type delegate_to: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
     template_fields = [
         '_project_id',
         '_model_name',
+        '_impersonation_chain',
     ]
 
     @apply_defaults
@@ -802,6 +939,7 @@ class MLEngineListVersionsOperator(BaseOperator):
                  project_id: Optional[str] = None,
                  gcp_conn_id: str = 'google_cloud_default',
                  delegate_to: Optional[str] = None,
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
 
         super().__init__(**kwargs)
@@ -809,6 +947,7 @@ class MLEngineListVersionsOperator(BaseOperator):
         self._model_name = model_name
         self._gcp_conn_id = gcp_conn_id
         self._delegate_to = delegate_to
+        self._impersonation_chain = impersonation_chain
         self._validate_inputs()
 
     def _validate_inputs(self):
@@ -816,7 +955,11 @@ class MLEngineListVersionsOperator(BaseOperator):
             raise AirflowException("The model_name parameter could not be empty.")
 
     def execute(self, context):
-        hook = MLEngineHook(gcp_conn_id=self._gcp_conn_id, delegate_to=self._delegate_to)
+        hook = MLEngineHook(
+            gcp_conn_id=self._gcp_conn_id,
+            delegate_to=self._delegate_to,
+            impersonation_chain=self._impersonation_chain,
+        )
 
         return hook.list_versions(
             project_id=self._project_id,
@@ -845,15 +988,25 @@ class MLEngineDeleteVersionOperator(BaseOperator):
     :type project_id: str
     :param gcp_conn_id: The connection ID to use when fetching connection info.
     :type gcp_conn_id: str
-    :param delegate_to: The account to impersonate, if any.
-        For this to work, the service account making the request must have
+    :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.
     :type delegate_to: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
     template_fields = [
         '_project_id',
         '_model_name',
         '_version_name',
+        '_impersonation_chain',
     ]
 
     @apply_defaults
@@ -864,6 +1017,7 @@ class MLEngineDeleteVersionOperator(BaseOperator):
                  project_id: Optional[str] = None,
                  gcp_conn_id: str = 'google_cloud_default',
                  delegate_to: Optional[str] = None,
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
 
         super().__init__(**kwargs)
@@ -872,6 +1026,7 @@ class MLEngineDeleteVersionOperator(BaseOperator):
         self._version_name = version_name
         self._gcp_conn_id = gcp_conn_id
         self._delegate_to = delegate_to
+        self._impersonation_chain = impersonation_chain
         self._validate_inputs()
 
     def _validate_inputs(self):
@@ -882,7 +1037,11 @@ class MLEngineDeleteVersionOperator(BaseOperator):
             raise AirflowException("The version_name parameter could not be empty.")
 
     def execute(self, context):
-        hook = MLEngineHook(gcp_conn_id=self._gcp_conn_id, delegate_to=self._delegate_to)
+        hook = MLEngineHook(
+            gcp_conn_id=self._gcp_conn_id,
+            delegate_to=self._delegate_to,
+            impersonation_chain=self._impersonation_chain,
+        )
 
         return hook.delete_version(
             project_id=self._project_id,
@@ -908,6 +1067,7 @@ class AIPlatformConsoleLink(BaseOperatorLink):
         return console_link
 
 
+# pylint: disable=too-many-instance-attributes
 class MLEngineStartTrainingJobOperator(BaseOperator):
     """
     Operator for launching a MLEngine training job.
@@ -950,8 +1110,8 @@ class MLEngineStartTrainingJobOperator(BaseOperator):
     :type project_id: str
     :param gcp_conn_id: The connection ID to use when fetching connection info.
     :type gcp_conn_id: str
-    :param delegate_to: The account to impersonate, if any.
-        For this to work, the service account making the request must have
+    :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.
     :type delegate_to: str
     :param mode: Can be one of 'DRY_RUN'/'CLOUD'. In 'DRY_RUN' mode, no real
@@ -961,6 +1121,15 @@ class MLEngineStartTrainingJobOperator(BaseOperator):
     :type mode: str
     :param labels: a dictionary containing labels for the job; passed to BigQuery
     :type labels: Dict[str, str]
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = [
@@ -974,7 +1143,8 @@ class MLEngineStartTrainingJobOperator(BaseOperator):
         '_master_type',
         '_runtime_version',
         '_python_version',
-        '_job_dir'
+        '_job_dir',
+        '_impersonation_chain',
     ]
 
     operator_extra_links = (
@@ -999,6 +1169,7 @@ class MLEngineStartTrainingJobOperator(BaseOperator):
                  delegate_to: Optional[str] = None,
                  mode: str = 'PRODUCTION',
                  labels: Optional[Dict[str, str]] = None,
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         super().__init__(**kwargs)
         self._project_id = project_id
@@ -1016,6 +1187,7 @@ class MLEngineStartTrainingJobOperator(BaseOperator):
         self._delegate_to = delegate_to
         self._mode = mode
         self._labels = labels
+        self._impersonation_chain = impersonation_chain
 
         if not self._project_id:
             raise AirflowException('Google Cloud project id is required.')
@@ -1070,7 +1242,10 @@ class MLEngineStartTrainingJobOperator(BaseOperator):
             return
 
         hook = MLEngineHook(
-            gcp_conn_id=self._gcp_conn_id, delegate_to=self._delegate_to)
+            gcp_conn_id=self._gcp_conn_id,
+            delegate_to=self._delegate_to,
+            impersonation_chain=self._impersonation_chain,
+        )
 
         # Helper method to check if the existing job's training input is the
         # same as the request we get here.
@@ -1113,15 +1288,25 @@ class MLEngineTrainingCancelJobOperator(BaseOperator):
     :type project_id: str
     :param gcp_conn_id: The connection ID to use when fetching connection info.
     :type gcp_conn_id: str
-    :param delegate_to: The account to impersonate, if any.
-        For this to work, the service account making the request must have
+    :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.
     :type delegate_to: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
 
     template_fields = [
         '_project_id',
         '_job_id',
+        '_impersonation_chain',
     ]
 
     @apply_defaults
@@ -1131,12 +1316,14 @@ class MLEngineTrainingCancelJobOperator(BaseOperator):
                  project_id: Optional[str] = None,
                  gcp_conn_id: str = 'google_cloud_default',
                  delegate_to: Optional[str] = None,
+                 impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
                  **kwargs) -> None:
         super().__init__(**kwargs)
         self._project_id = project_id
         self._job_id = job_id
         self._gcp_conn_id = gcp_conn_id
         self._delegate_to = delegate_to
+        self._impersonation_chain = impersonation_chain
 
         if not self._project_id:
             raise AirflowException('Google Cloud project id is required.')
@@ -1145,7 +1332,8 @@ class MLEngineTrainingCancelJobOperator(BaseOperator):
 
         hook = MLEngineHook(
             gcp_conn_id=self._gcp_conn_id,
-            delegate_to=self._delegate_to
+            delegate_to=self._delegate_to,
+            impersonation_chain=self._impersonation_chain,
         )
 
         hook.cancel_job(project_id=self._project_id, job_id=_normalize_mlengine_job_id(self._job_id))
diff --git a/airflow/providers/google/cloud/operators/natural_language.py b/airflow/providers/google/cloud/operators/natural_language.py
index a314dde..eed2ee5 100644
--- a/airflow/providers/google/cloud/operators/natural_language.py
+++ b/airflow/providers/google/cloud/operators/natural_language.py
@@ -55,9 +55,18 @@ class CloudNaturalLanguageAnalyzeEntitiesOperator(BaseOperator):
     :type metadata: Sequence[Tuple[str, str]]
     :param gcp_conn_id: The connection ID to use connecting to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
     # [START natural_language_analyze_entities_template_fields]
-    template_fields = ("document", "gcp_conn_id")
+    template_fields = ("document", "gcp_conn_id", "impersonation_chain",)
     # [END natural_language_analyze_entities_template_fields]
 
     @apply_defaults
@@ -69,6 +78,7 @@ class CloudNaturalLanguageAnalyzeEntitiesOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[MetaData] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -78,9 +88,13 @@ class CloudNaturalLanguageAnalyzeEntitiesOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudNaturalLanguageHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudNaturalLanguageHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
 
         self.log.info("Start analyzing entities")
         response = hook.analyze_entities(
@@ -114,10 +128,20 @@ class CloudNaturalLanguageAnalyzeEntitySentimentOperator(BaseOperator):
     :type metadata: Sequence[Tuple[str, str]]]
     :param gcp_conn_id: The connection ID to use connecting to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
+
     :rtype: google.cloud.language_v1.types.AnalyzeEntitiesResponse
     """
     # [START natural_language_analyze_entity_sentiment_template_fields]
-    template_fields = ("document", "gcp_conn_id")
+    template_fields = ("document", "gcp_conn_id", "impersonation_chain",)
     # [END natural_language_analyze_entity_sentiment_template_fields]
 
     @apply_defaults
@@ -129,6 +153,7 @@ class CloudNaturalLanguageAnalyzeEntitySentimentOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[MetaData] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -138,9 +163,13 @@ class CloudNaturalLanguageAnalyzeEntitySentimentOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudNaturalLanguageHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudNaturalLanguageHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
 
         self.log.info("Start entity sentiment analyze")
         response = hook.analyze_entity_sentiment(
@@ -177,10 +206,20 @@ class CloudNaturalLanguageAnalyzeSentimentOperator(BaseOperator):
     :type metadata: sequence[tuple[str, str]]]
     :param gcp_conn_id: The connection ID to use connecting to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
+
     :rtype: google.cloud.language_v1.types.AnalyzeEntitiesResponse
     """
     # [START natural_language_analyze_sentiment_template_fields]
-    template_fields = ("document", "gcp_conn_id")
+    template_fields = ("document", "gcp_conn_id", "impersonation_chain",)
     # [END natural_language_analyze_sentiment_template_fields]
 
     @apply_defaults
@@ -192,6 +231,7 @@ class CloudNaturalLanguageAnalyzeSentimentOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[MetaData] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -201,9 +241,13 @@ class CloudNaturalLanguageAnalyzeSentimentOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudNaturalLanguageHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudNaturalLanguageHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
 
         self.log.info("Start sentiment analyze")
         response = hook.analyze_sentiment(
@@ -234,9 +278,18 @@ class CloudNaturalLanguageClassifyTextOperator(BaseOperator):
     :type metadata: sequence[tuple[str, str]]]
     :param gcp_conn_id: The connection ID to use connecting to Google Cloud Platform.
     :type gcp_conn_id: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
     # [START natural_language_classify_text_template_fields]
-    template_fields = ("document", "gcp_conn_id")
+    template_fields = ("document", "gcp_conn_id", "impersonation_chain",)
     # [END natural_language_classify_text_template_fields]
 
     @apply_defaults
@@ -247,6 +300,7 @@ class CloudNaturalLanguageClassifyTextOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[MetaData] = None,
         gcp_conn_id: str = "google_cloud_default",
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         super().__init__(**kwargs)
@@ -255,9 +309,13 @@ class CloudNaturalLanguageClassifyTextOperator(BaseOperator):
         self.timeout = timeout
         self.metadata = metadata
         self.gcp_conn_id = gcp_conn_id
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = CloudNaturalLanguageHook(gcp_conn_id=self.gcp_conn_id)
+        hook = CloudNaturalLanguageHook(
+            gcp_conn_id=self.gcp_conn_id,
+            impersonation_chain=self.impersonation_chain,
+        )
 
         self.log.info("Start text classify")
         response = hook.classify_text(
diff --git a/airflow/providers/google/cloud/operators/pubsub.py b/airflow/providers/google/cloud/operators/pubsub.py
index 2ec8b08..425b8c6 100644
--- a/airflow/providers/google/cloud/operators/pubsub.py
+++ b/airflow/providers/google/cloud/operators/pubsub.py
@@ -76,9 +76,9 @@ class PubSubCreateTopicOperator(BaseOperator):
     :param gcp_conn_id: The connection ID to use connecting to
         Google Cloud Platform.
     :type gcp_conn_id: str
-    :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 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.
     :type delegate_to: str
     :param labels: Client-assigned labels; see
         https://cloud.google.com/pubsub/docs/labels
@@ -105,8 +105,17 @@ class PubSubCreateTopicOperator(BaseOperator):
     :type metadata: Sequence[Tuple[str, str]]]
     :param project: (Deprecated) the GCP project ID where the topic will be created
     :type project: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
-    template_fields = ['project_id', 'topic']
+    template_fields = ['project_id', 'topic', 'impersonation_chain', ]
     ui_color = '#0273d4'
 
     # pylint: disable=too-many-arguments
@@ -125,6 +134,7 @@ class PubSubCreateTopicOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         project: Optional[str] = None,
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         # To preserve backward compatibility
@@ -147,10 +157,14 @@ class PubSubCreateTopicOperator(BaseOperator):
         self.retry = retry
         self.timeout = timeout
         self.metadata = metadata
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = PubSubHook(gcp_conn_id=self.gcp_conn_id,
-                          delegate_to=self.delegate_to)
+        hook = PubSubHook(
+            gcp_conn_id=self.gcp_conn_id,
+            delegate_to=self.delegate_to,
+            impersonation_chain=self.impersonation_chain,
+        )
 
         self.log.info("Creating topic %s", self.topic)
         hook.create_topic(
@@ -237,9 +251,9 @@ class PubSubCreateSubscriptionOperator(BaseOperator):
     :param gcp_conn_id: The connection ID to use connecting to
         Google Cloud Platform.
     :type gcp_conn_id: str
-    :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 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.
     :type delegate_to: str
     :param push_config: If push delivery is used with this subscription,
         this field is used to configure it. An empty ``pushConfig`` signifies
@@ -301,11 +315,21 @@ class PubSubCreateSubscriptionOperator(BaseOperator):
     :param subscription_project: (Deprecated) the GCP project ID where the subscription
         will be created. If empty, ``topic_project`` will be used.
     :type subscription_project: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
-    template_fields = ['project_id', 'topic', 'subscription', 'subscription_project_id']
+    template_fields = ['project_id', 'topic', 'subscription', 'subscription_project_id',
+                       'impersonation_chain', ]
     ui_color = '#0273d4'
 
-    # pylint: disable=too-many-arguments
+    # pylint: disable=too-many-arguments, too-many-locals
     @apply_defaults
     def __init__(
         self, *,
@@ -331,6 +355,7 @@ class PubSubCreateSubscriptionOperator(BaseOperator):
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         topic_project: Optional[str] = None,
         subscription_project: Optional[str] = None,
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
 
@@ -368,10 +393,14 @@ class PubSubCreateSubscriptionOperator(BaseOperator):
         self.retry = retry
         self.timeout = timeout
         self.metadata = metadata
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = PubSubHook(gcp_conn_id=self.gcp_conn_id,
-                          delegate_to=self.delegate_to)
+        hook = PubSubHook(
+            gcp_conn_id=self.gcp_conn_id,
+            delegate_to=self.delegate_to,
+            impersonation_chain=self.impersonation_chain,
+        )
 
         self.log.info("Creating subscription for topic %s", self.topic)
         result = hook.create_subscription(
@@ -441,9 +470,9 @@ class PubSubDeleteTopicOperator(BaseOperator):
     :param gcp_conn_id: The connection ID to use connecting to
         Google Cloud Platform.
     :type gcp_conn_id: str
-    :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 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.
     :type delegate_to: str
     :param retry: (Optional) A retry object used to retry requests.
         If None is specified, requests will not be retried.
@@ -456,8 +485,17 @@ class PubSubDeleteTopicOperator(BaseOperator):
     :type metadata: Sequence[Tuple[str, str]]]
     :param project: (Deprecated) the GCP project ID where the topic will be created
     :type project: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
-    template_fields = ['project_id', 'topic']
+    template_fields = ['project_id', 'topic', 'impersonation_chain', ]
     ui_color = '#cb4335'
 
     @apply_defaults
@@ -472,6 +510,7 @@ class PubSubDeleteTopicOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         project: Optional[str] = None,
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         # To preserve backward compatibility
@@ -491,10 +530,14 @@ class PubSubDeleteTopicOperator(BaseOperator):
         self.retry = retry
         self.timeout = timeout
         self.metadata = metadata
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = PubSubHook(gcp_conn_id=self.gcp_conn_id,
-                          delegate_to=self.delegate_to)
+        hook = PubSubHook(
+            gcp_conn_id=self.gcp_conn_id,
+            delegate_to=self.delegate_to,
+            impersonation_chain=self.impersonation_chain,
+        )
 
         self.log.info("Deleting topic %s", self.topic)
         hook.delete_topic(
@@ -552,9 +595,9 @@ class PubSubDeleteSubscriptionOperator(BaseOperator):
     :param gcp_conn_id: The connection ID to use connecting to
         Google Cloud Platform.
     :type gcp_conn_id: str
-    :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 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.
     :type delegate_to: str
     :param retry: (Optional) A retry object used to retry requests.
         If None is specified, requests will not be retried.
@@ -567,8 +610,17 @@ class PubSubDeleteSubscriptionOperator(BaseOperator):
     :type metadata: Sequence[Tuple[str, str]]]
     :param project: (Deprecated) the GCP project ID where the topic will be created
     :type project: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
-    template_fields = ['project_id', 'subscription']
+    template_fields = ['project_id', 'subscription', 'impersonation_chain', ]
     ui_color = '#cb4335'
 
     @apply_defaults
@@ -583,6 +635,7 @@ class PubSubDeleteSubscriptionOperator(BaseOperator):
         timeout: Optional[float] = None,
         metadata: Optional[Sequence[Tuple[str, str]]] = None,
         project: Optional[str] = None,
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         # To preserve backward compatibility
@@ -602,10 +655,14 @@ class PubSubDeleteSubscriptionOperator(BaseOperator):
         self.retry = retry
         self.timeout = timeout
         self.metadata = metadata
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = PubSubHook(gcp_conn_id=self.gcp_conn_id,
-                          delegate_to=self.delegate_to)
+        hook = PubSubHook(
+            gcp_conn_id=self.gcp_conn_id,
+            delegate_to=self.delegate_to,
+            impersonation_chain=self.impersonation_chain,
+        )
 
         self.log.info("Deleting subscription %s", self.subscription)
         hook.delete_subscription(
@@ -665,14 +722,23 @@ class PubSubPublishMessageOperator(BaseOperator):
     :param gcp_conn_id: The connection ID to use connecting to
         Google Cloud Platform.
     :type gcp_conn_id: str
-    :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 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.
     :type delegate_to: str
     :param project: (Deprecated) the GCP project ID where the topic will be created
     :type project: str
+    :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.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        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).
+    :type impersonation_chain: Union[str, Sequence[str]]
     """
-    template_fields = ['project_id', 'topic', 'messages']
+    template_fields = ['project_id', 'topic', 'messages', 'impersonation_chain', ]
     ui_color = '#0273d4'
 
     @apply_defaults
@@ -684,6 +750,7 @@ class PubSubPublishMessageOperator(BaseOperator):
         gcp_conn_id: str = 'google_cloud_default',
         delegate_to: Optional[str] = None,
         project: Optional[str] = None,
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
         **kwargs
     ) -> None:
         # To preserve backward compatibility
@@ -700,10 +767,14 @@ class PubSubPublishMessageOperator(BaseOperator):
         self.messages = messages
         self.gcp_conn_id = gcp_conn_id
         self.delegate_to = delegate_to
+        self.impersonation_chain = impersonation_chain
 
     def execute(self, context):
-        hook = PubSubHook(gcp_conn_id=self.gcp_conn_id,
-                          delegate_to=self.delegate_to)
+        hook = PubSubHook(
+            gcp_conn_id=self.gcp_conn_id,
+            delegate_to=self.delegate_to,
+            impersonation_chain=self.impersonation_chain,
+        )
 
         self.log.info("Publishing to topic %s", self.topic)
         hook.publish(project_id=self.project_id, topic=self.topic, messages=self.messages)
@@ -746,9 +817,9 @@ class PubSubPullOperator(BaseOperator):
     :param gcp_conn_id: The connection ID to use connecting to
         Google Cloud Platform.
     :type gcp_conn_id: str
-    :param delegate_to: The account to impersonate, if any.
-        For this to work, the service account making the request
... 11199 lines suppressed ...