diff --git a/airflow/providers/google/cloud/utils/credentials_provider.py b/airflow/providers/google/cloud/utils/credentials_provider.py index ef9f50743f8ad..22c6de17d57c9 100644 --- a/airflow/providers/google/cloud/utils/credentials_provider.py +++ b/airflow/providers/google/cloud/utils/credentials_provider.py @@ -35,6 +35,9 @@ from airflow.exceptions import AirflowException from airflow.providers.google.cloud._internal_client.secret_manager_client import _SecretManagerClient +from airflow.providers.google.cloud.utils.external_token_supplier import ( + ClientCredentialsGrantFlowTokenSupplier, +) from airflow.utils.log.logging_mixin import LoggingMixin from airflow.utils.process_utils import patch_environ @@ -210,6 +213,10 @@ def __init__( target_principal: str | None = None, delegates: Sequence[str] | None = None, is_anonymous: bool | None = None, + idp_issuer_url: str | None = None, + client_id: str | None = None, + client_secret: str | None = None, + idp_extra_params_dict: dict[str, str] | None = None, ) -> None: super().__init__() key_options = [key_path, keyfile_dict, credential_config_file, key_secret_name, is_anonymous] @@ -229,6 +236,10 @@ def __init__( self.target_principal = target_principal self.delegates = delegates self.is_anonymous = is_anonymous + self.idp_issuer_url = idp_issuer_url + self.client_id = client_id + self.client_secret = client_secret + self.idp_extra_params_dict = idp_extra_params_dict def get_credentials_and_project(self) -> tuple[Credentials, str]: """ @@ -248,6 +259,10 @@ def get_credentials_and_project(self) -> tuple[Credentials, str]: credentials, project_id = self._get_credentials_using_key_secret_name() elif self.keyfile_dict: credentials, project_id = self._get_credentials_using_keyfile_dict() + elif self.idp_issuer_url: + credentials, project_id = ( + self._get_credentials_using_credential_config_file_and_token_supplier() + ) elif self.credential_config_file: credentials, project_id = self._get_credentials_using_credential_config_file() else: @@ -357,6 +372,24 @@ def _get_credentials_using_credential_config_file(self) -> tuple[Credentials, st return credentials, project_id + def _get_credentials_using_credential_config_file_and_token_supplier(self): + self._log_info( + "Getting connection using credential configuration file and external Identity Provider." + ) + + if not self.credential_config_file: + raise AirflowException( + "Credential Configuration File is needed to use authentication by External Identity Provider." + ) + + info = _get_info_from_credential_configuration_file(self.credential_config_file) + info["subject_token_supplier"] = ClientCredentialsGrantFlowTokenSupplier( + oidc_issuer_url=self.idp_issuer_url, client_id=self.client_id, client_secret=self.client_secret + ) + + credentials, project_id = google.auth.load_credentials_from_dict(info=info, scopes=self.scopes) + return credentials, project_id + def _get_credentials_using_adc(self) -> tuple[Credentials, str]: self._log_info( "Getting connection using `google.auth.default()` since no explicit credentials are provided." @@ -426,3 +459,38 @@ def _get_project_id_from_service_account_email(service_account_email: str) -> st raise AirflowException( f"Could not extract project_id from service account's email: {service_account_email}." ) + + +def _get_info_from_credential_configuration_file( + credential_configuration_file: str | dict[str, str], +) -> dict[str, str]: + """ + Extract the Credential Configuration File information, either from a json file, json string or dictionary. + + :param credential_configuration_file: File path or content (as json string or dictionary) of a GCP credential configuration file. + + :return: Returns a dictionary containing the Credential Configuration File information. + """ + # if it's already a dict, just return it + if isinstance(credential_configuration_file, dict): + return credential_configuration_file + + if not isinstance(credential_configuration_file, str): + raise AirflowException( + f"Invalid argument type, expected str or dict, got {type(credential_configuration_file)}." + ) + + if os.path.exists(credential_configuration_file): # attempts to load from json file + with open(credential_configuration_file) as file_obj: + try: + return json.load(file_obj) + except ValueError: + raise AirflowException( + f"Credential Configuration File '{credential_configuration_file}' is not a valid json file." + ) + + # if not a file, attempt to load it from a json string + try: + return json.loads(credential_configuration_file) + except ValueError: + raise AirflowException("Credential Configuration File is not a valid json string.") diff --git a/airflow/providers/google/cloud/utils/external_token_supplier.py b/airflow/providers/google/cloud/utils/external_token_supplier.py new file mode 100644 index 0000000000000..6abdcfeeae42c --- /dev/null +++ b/airflow/providers/google/cloud/utils/external_token_supplier.py @@ -0,0 +1,175 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +from __future__ import annotations + +import abc +import time +from functools import wraps +from typing import TYPE_CHECKING, Any + +import requests +from google.auth.exceptions import RefreshError +from google.auth.identity_pool import SubjectTokenSupplier + +if TYPE_CHECKING: + from google.auth.external_account import SupplierContext + from google.auth.transport import Request + +from airflow.utils.log.logging_mixin import LoggingMixin + + +def cache_token_decorator(get_subject_token_method): + """Cache calls to ``SubjectTokenSupplier`` instances' ``get_token_supplier`` methods. + + Different instances of a same SubjectTokenSupplier class with the same attributes + share the OIDC token cache. + + :param get_subject_token_method: A method that returns both a token and an integer specifying + the time in seconds until the token expires + + See also: + https://googleapis.dev/python/google-auth/latest/reference/google.auth.identity_pool.html#google.auth.identity_pool.SubjectTokenSupplier.get_subject_token + """ + cache = {} + + @wraps(get_subject_token_method) + def wrapper(supplier_instance: CacheTokenSupplier, *args, **kwargs) -> str: + """Obeys the interface set by ``SubjectTokenSupplier`` for ``get_subject_token`` methods. + + :param supplier_instance: the SubjectTokenSupplier instance whose get_subject_token method is being decorated + :return: The token string + """ + nonlocal cache + + cache_key = supplier_instance.get_subject_key() + token: dict[str, str | float] = {} + + if cache_key not in cache or cache[cache_key]["expiration_time"] < time.monotonic(): + supplier_instance.log.info("OIDC token missing or expired") + try: + access_token, expires_in = get_subject_token_method(supplier_instance, *args, **kwargs) + if not isinstance(expires_in, int) or not isinstance(access_token, str): + raise RefreshError # assume error if strange values are provided + + except RefreshError: + supplier_instance.log.error("Failed retrieving new OIDC Token from IdP") + raise + + expiration_time = time.monotonic() + float(expires_in) + token["access_token"] = access_token + token["expiration_time"] = expiration_time + cache[cache_key] = token + + supplier_instance.log.info("New OIDC token retrieved, expires in %s seconds.", expires_in) + + return cache[cache_key]["access_token"] + + return wrapper + + +class CacheTokenSupplier(LoggingMixin, SubjectTokenSupplier): + """ + A superclass for all Subject Token Supplier classes that wish to implement a caching mechanism. + + Child classes must implement the ``get_subject_key`` method to generate a string that serves as the cache key, + ensuring that tokens are shared appropriately among instances. + + Methods: + get_subject_key: Abstract method to be implemented by child classes. It should return a string that serves as the cache key. + """ + + def __init__(self): + super().__init__() + + @abc.abstractmethod + def get_subject_key(self) -> str: + raise NotImplementedError("") + + +class ClientCredentialsGrantFlowTokenSupplier(CacheTokenSupplier): + """ + Class that retrieves an OIDC token from an external IdP using OAuth2.0 Client Credentials Grant flow. + + This class implements the ``SubjectTokenSupplier`` interface class used by ``google.auth.identity_pool.Credentials`` + + :params oidc_issuer_url: URL of the IdP that performs OAuth2.0 Client Credentials Grant flow and returns an OIDC token. + :params client_id: Client ID of the application requesting the token + :params client_secret: Client secret of the application requesting the token + :params extra_params_kwargs: Extra parameters to be passed in the payload of the POST request to the `oidc_issuer_url` + + See also: + https://googleapis.dev/python/google-auth/latest/reference/google.auth.identity_pool.html#google.auth.identity_pool.SubjectTokenSupplier + """ + + def __init__( + self, + oidc_issuer_url: str, + client_id: str, + client_secret: str, + **extra_params_kwargs: Any, + ) -> None: + super().__init__() + self.oidc_issuer_url = oidc_issuer_url + self.client_id = client_id + self.client_secret = client_secret + self.extra_params_kwargs = extra_params_kwargs + + @cache_token_decorator + def get_subject_token(self, context: SupplierContext, request: Request) -> tuple[str, int]: + """Perform Client Credentials Grant flow with IdP and retrieves an OIDC token and expiration time.""" + self.log.info("Requesting new OIDC token from external IdP.") + try: + response = requests.post( + self.oidc_issuer_url, + data={ + "grant_type": "client_credentials", + "client_id": self.client_id, + "client_secret": self.client_secret, + **self.extra_params_kwargs, + }, + ) + response.raise_for_status() + except requests.HTTPError as e: + raise RefreshError(str(e)) + except requests.ConnectionError as e: + raise RefreshError(str(e)) + + try: + response_dict = response.json() + except requests.JSONDecodeError: + raise RefreshError(f"Didn't get a json response from {self.oidc_issuer_url}") + + # These fields are required + if {"access_token", "expires_in"} - set(response_dict.keys()): + # TODO more information about the error can be provided in the exception by inspecting the response + raise RefreshError(f"No access token returned from {self.oidc_issuer_url}") + + return response_dict["access_token"], response_dict["expires_in"] + + def get_subject_key(self) -> str: + """ + Create a cache key using the OIDC issuer URL, client ID, client secret and additional parameters. + + Instances with the same credentials will share tokens. + """ + cache_key = ( + self.oidc_issuer_url + + self.client_id + + self.client_secret + + ",".join(sorted(self.extra_params_kwargs)) + ) + return cache_key diff --git a/airflow/providers/google/common/hooks/base_google.py b/airflow/providers/google/common/hooks/base_google.py index 04c86f378c112..97779fa695e10 100644 --- a/airflow/providers/google/common/hooks/base_google.py +++ b/airflow/providers/google/common/hooks/base_google.py @@ -248,6 +248,20 @@ def get_connection_form_widgets(cls) -> dict[str, Any]: "impersonation_chain": StringField( lazy_gettext("Impersonation Chain"), widget=BS3TextFieldWidget() ), + "idp_issuer_url": StringField( + lazy_gettext("IdP Token Issue URL (Client Credentials Grant Flow)"), + widget=BS3TextFieldWidget(), + ), + "client_id": StringField( + lazy_gettext("Client ID (Client Credentials Grant Flow)"), widget=BS3TextFieldWidget() + ), + "client_secret": StringField( + lazy_gettext("Client Secret (Client Credentials Grant Flow)"), + widget=BS3PasswordFieldWidget(), + ), + "idp_extra_parameters": StringField( + lazy_gettext("IdP Extra Request Parameters"), widget=BS3TextFieldWidget() + ), "is_anonymous": BooleanField( lazy_gettext("Anonymous credentials (ignores all other settings)"), default=False ), @@ -305,6 +319,18 @@ def get_credentials_and_project_id(self) -> tuple[Credentials, str | None]: target_principal, delegates = _get_target_principal_and_delegates(self.impersonation_chain) is_anonymous = self._get_field("is_anonymous") + idp_issuer_url: str | None = self._get_field("idp_issuer_url", None) + client_id: str | None = self._get_field("client_id", None) + client_secret: str | None = self._get_field("client_secret", None) + idp_extra_params: str | None = self._get_field("idp_extra_params", None) + + idp_extra_params_dict: dict[str, str] | None = None + if idp_extra_params: + try: + idp_extra_params_dict = json.loads(idp_extra_params) + except json.decoder.JSONDecodeError: + raise AirflowException("Invalid JSON.") + credentials, project_id = get_credentials_and_project_id( key_path=key_path, keyfile_dict=keyfile_dict_json, @@ -316,6 +342,10 @@ def get_credentials_and_project_id(self) -> tuple[Credentials, str | None]: target_principal=target_principal, delegates=delegates, is_anonymous=is_anonymous, + idp_issuer_url=idp_issuer_url, + client_id=client_id, + client_secret=client_secret, + idp_extra_params_dict=idp_extra_params_dict, ) overridden_project_id = self._get_field("project") diff --git a/tests/providers/google/cloud/utils/test_credentials_provider.py b/tests/providers/google/cloud/utils/test_credentials_provider.py index 2fdf756e2803d..3cab06cee8a46 100644 --- a/tests/providers/google/cloud/utils/test_credentials_provider.py +++ b/tests/providers/google/cloud/utils/test_credentials_provider.py @@ -54,6 +54,26 @@ ACCOUNT_3_ANOTHER_PROJECT = "account_3@another_project_id.iam.gserviceaccount.com" ANOTHER_PROJECT_ID = "another_project_id" CRED_PROVIDER_LOGGER_NAME = "airflow.providers.google.cloud.utils.credentials_provider._CredentialProvider" +IDP_LINK = "http://example.com/idp" +CLIENT_ID = "your-client-id" +CLIENT_SECRET = "your-client-secret" +TEST_AUDIENCE = "test-audience" +TOKEN_TYPE = "urn:ietf:params:oauth:token-type:jwt" +ACCOUNT_IMPERSONATION = "http://example.com/impersonate" +CREDENTIAL_CONFIG_STRING_FILE = ( + '{"audience": "' + + TEST_AUDIENCE + + '", "subject_token_type": "' + + TOKEN_TYPE + + '", "service_account_impersonation_url": "' + + ACCOUNT_IMPERSONATION + + '"}' +) +CREDENTIAL_CONFIG_DICT_FILE = { + "audience": TEST_AUDIENCE, + "subject_token_type": TOKEN_TYPE, + "service_account_impersonation_url": ACCOUNT_IMPERSONATION, +} @pytest.fixture @@ -411,6 +431,93 @@ def test_disable_logging(self, mock_default, mock_info, mock_file, assert_no_log disable_logging=True, ) + @mock.patch("google.auth.load_credentials_from_dict", return_value=("CREDENTIALS", "PROJECT_ID")) + def test_get_credentials_using_identity_provider(self, mock_load_credentials_from_file, caplog): + with caplog.at_level(level=logging.DEBUG, logger=CRED_PROVIDER_LOGGER_NAME): + caplog.clear() + result = get_credentials_and_project_id( + credential_config_file=CREDENTIAL_CONFIG_STRING_FILE, + idp_issuer_url=IDP_LINK, + client_id=CLIENT_ID, + client_secret=CLIENT_SECRET, + ) + mock_load_credentials_from_file.assert_called_once_with( + info={ + "audience": TEST_AUDIENCE, + "subject_token_type": TOKEN_TYPE, + "service_account_impersonation_url": ACCOUNT_IMPERSONATION, + "subject_token_supplier": ANY, + }, + scopes=ANY, + ) + assert result == ("CREDENTIALS", "PROJECT_ID") + assert ( + "Getting connection using credential configuration file and external Identity Provider." + in caplog.messages + ) + + @mock.patch("google.auth.load_credentials_from_dict", return_value=("CREDENTIALS", "PROJECT_ID")) + def test_get_credentials_using_idp_dict_config_file(self, mock_load_credentials_from_file, caplog): + with caplog.at_level(level=logging.DEBUG, logger=CRED_PROVIDER_LOGGER_NAME): + caplog.clear() + result = get_credentials_and_project_id( + credential_config_file=CREDENTIAL_CONFIG_DICT_FILE, + idp_issuer_url=IDP_LINK, + client_id=CLIENT_ID, + client_secret=CLIENT_SECRET, + ) + mock_load_credentials_from_file.assert_called_once_with( + info={ + "audience": TEST_AUDIENCE, + "subject_token_type": TOKEN_TYPE, + "service_account_impersonation_url": ACCOUNT_IMPERSONATION, + "subject_token_supplier": ANY, + }, + scopes=ANY, + ) + assert result == ("CREDENTIALS", "PROJECT_ID") + assert ( + "Getting connection using credential configuration file and external Identity Provider." + in caplog.messages + ) + + def test_get_credentials_using_idp_no_credential_config(self): + with pytest.raises( + AirflowException, + match=re.escape( + "Credential Configuration File is needed to use authentication by External Identity Provider." + ), + ): + get_credentials_and_project_id( + idp_issuer_url=IDP_LINK, + client_id=CLIENT_ID, + client_secret=CLIENT_SECRET, + ) + + def test_get_credentials_using_idp_invalid_json_credential_config(self): + with pytest.raises( + AirflowException, + match=re.escape("Credential Configuration File is not a valid json string."), + ): + get_credentials_and_project_id( + credential_config_file="invalid json}}}}", + idp_issuer_url=IDP_LINK, + client_id=CLIENT_ID, + client_secret=CLIENT_SECRET, + ) + + def test_get_credentials_using_idp_invalid_format_credential_config(self): + with pytest.raises( + AirflowException, + match=re.escape(f"Invalid argument type, expected str or dict, got {bool}."), + ): + get_credentials_and_project_id( + credential_config_file=True, + idp_issuer_url=IDP_LINK, + client_id=CLIENT_ID, + client_secret=CLIENT_SECRET, + ) + class TestGetScopes: def test_get_scopes_with_default(self): diff --git a/tests/providers/google/cloud/utils/test_external_token_supplier.py b/tests/providers/google/cloud/utils/test_external_token_supplier.py new file mode 100644 index 0000000000000..5574a4fb5d011 --- /dev/null +++ b/tests/providers/google/cloud/utils/test_external_token_supplier.py @@ -0,0 +1,135 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +from __future__ import annotations + +from unittest.mock import ANY + +import pytest +import requests_mock +from google.auth.exceptions import RefreshError + +from airflow.providers.google.cloud.utils.external_token_supplier import ( + ClientCredentialsGrantFlowTokenSupplier, +) + +MOCK_URL1 = "http://mock-idp/token1" +MOCK_URL2 = "http://mock-idp/token2" +MOCK_URL3 = "http://mock-idp/token3" +MOCK_URL4 = "http://mock-idp/token4" +CLIENT_ID = "test-client-id" +CLIENT_ID2 = "test-client-id2" +CLIENT_ID3 = "test-client-id3" +CLIENT_SECRET = "test-client-secret" +CLIENT_SECRET2 = "test-client-secret2" + + +class TestClientCredentialsGrantFlowTokenSupplier: + def test_get_subject_token_success(self): + token_supplier = ClientCredentialsGrantFlowTokenSupplier( + oidc_issuer_url=MOCK_URL1, + client_id=CLIENT_ID, + client_secret=CLIENT_SECRET, + ) + + with requests_mock.Mocker() as m: + m.post(MOCK_URL1, json={"access_token": "mock-token", "expires_in": 3600}) + token = token_supplier.get_subject_token(ANY, ANY) + + assert token == "mock-token" + + def test_cache_token_decorator(self): + token_supplier = ClientCredentialsGrantFlowTokenSupplier( + oidc_issuer_url=MOCK_URL2, + client_id=CLIENT_ID, + client_secret=CLIENT_SECRET, + ) + + with requests_mock.Mocker() as m: + m.post(MOCK_URL2, json={"access_token": "mock-token", "expires_in": 3600}) + token = token_supplier.get_subject_token(ANY, ANY) + + assert token == "mock-token" + + # instances with same credentials and url should get previous token + token_supplier2 = ClientCredentialsGrantFlowTokenSupplier( + oidc_issuer_url=MOCK_URL2, + client_id=CLIENT_ID, + client_secret=CLIENT_SECRET, + ) + + with requests_mock.Mocker() as m2: + m2.post(MOCK_URL2, json={"access_token": "mock-token2", "expires_in": 3600}) + token = token_supplier2.get_subject_token(ANY, ANY) + + assert token == "mock-token" + + def test_cache_token_decorator_diff_credentials(self): + token_supplier = ClientCredentialsGrantFlowTokenSupplier( + oidc_issuer_url=MOCK_URL3, + client_id=CLIENT_ID, + client_secret=CLIENT_SECRET, + ) + + with requests_mock.Mocker() as m: + m.post(MOCK_URL3, json={"access_token": "mock-token", "expires_in": 3600}) + token = token_supplier.get_subject_token(ANY, ANY) + + assert token == "mock-token" + + # instances with different credentials and same url should get different tokens + token_supplier2 = ClientCredentialsGrantFlowTokenSupplier( + oidc_issuer_url=MOCK_URL3, + client_id=CLIENT_ID2, + client_secret=CLIENT_SECRET2, + ) + + with requests_mock.Mocker() as m2: + m2.post(MOCK_URL3, json={"access_token": "mock-token2", "expires_in": 3600}) + token = token_supplier2.get_subject_token(ANY, ANY) + + assert token == "mock-token2" + + def test_cache_token_expiration_date(self): + token_supplier = ClientCredentialsGrantFlowTokenSupplier( + oidc_issuer_url=MOCK_URL4, + client_id=CLIENT_ID, + client_secret=CLIENT_SECRET, + ) + + with requests_mock.Mocker() as m: + m.post(MOCK_URL4, json={"access_token": "mock-token", "expires_in": -1}) + token = token_supplier.get_subject_token(ANY, ANY) + + assert token == "mock-token" + + with requests_mock.Mocker() as m2: + m2.post(MOCK_URL4, json={"access_token": "mock-token2", "expires_in": 3600}) + token = token_supplier.get_subject_token(ANY, ANY) + + assert token == "mock-token2" + + def test_get_subject_token_failure(self): + token_supplier = ClientCredentialsGrantFlowTokenSupplier( + oidc_issuer_url=MOCK_URL4, + client_id=CLIENT_ID3, + client_secret=CLIENT_SECRET, + ) + with requests_mock.Mocker() as m: + m.post(MOCK_URL4, status_code=400) + + with pytest.raises(RefreshError): + token_supplier.get_subject_token(ANY, ANY) diff --git a/tests/providers/google/common/hooks/test_base_google.py b/tests/providers/google/common/hooks/test_base_google.py index 50874c1c3638d..2b67b8844ecbc 100644 --- a/tests/providers/google/common/hooks/test_base_google.py +++ b/tests/providers/google/common/hooks/test_base_google.py @@ -413,6 +413,10 @@ def test_get_credentials_and_project_id_with_default_auth(self, mock_get_creds_a target_principal=None, delegates=None, is_anonymous=None, + idp_issuer_url=None, + client_id=None, + client_secret=None, + idp_extra_params_dict=None, ) assert ("CREDENTIALS", "PROJECT_ID") == result @@ -451,6 +455,10 @@ def test_get_credentials_and_project_id_with_service_account_file(self, mock_get target_principal=None, delegates=None, is_anonymous=None, + idp_issuer_url=None, + client_id=None, + client_secret=None, + idp_extra_params_dict=None, ) assert (mock_credentials, "PROJECT_ID") == result @@ -482,6 +490,10 @@ def test_get_credentials_and_project_id_with_service_account_info(self, mock_get target_principal=None, delegates=None, is_anonymous=None, + idp_issuer_url=None, + client_id=None, + client_secret=None, + idp_extra_params_dict=None, ) assert (mock_credentials, "PROJECT_ID") == result @@ -503,6 +515,10 @@ def test_get_credentials_and_project_id_with_default_auth_and_delegate(self, moc target_principal=None, delegates=None, is_anonymous=None, + idp_issuer_url=None, + client_id=None, + client_secret=None, + idp_extra_params_dict=None, ) assert (mock_credentials, "PROJECT_ID") == result @@ -540,6 +556,10 @@ def test_get_credentials_and_project_id_with_default_auth_and_overridden_project target_principal=None, delegates=None, is_anonymous=None, + idp_issuer_url=None, + client_id=None, + client_secret=None, + idp_extra_params_dict=None, ) assert ("CREDENTIALS", "SECOND_PROJECT_ID") == result @@ -576,6 +596,10 @@ def test_get_credentials_and_project_id_with_is_anonymous(self, mock_get_creds_a target_principal=None, delegates=None, is_anonymous=True, + idp_issuer_url=None, + client_id=None, + client_secret=None, + idp_extra_params_dict=None, ) @pytest.mark.skipif( @@ -784,6 +808,10 @@ def test_get_credentials_and_project_id_with_impersonation_chain( target_principal=target_principal, delegates=delegates, is_anonymous=None, + idp_issuer_url=None, + client_id=None, + client_secret=None, + idp_extra_params_dict=None, ) assert (mock_credentials, PROJECT_ID) == result