From 286e38c3fbe3631cc46411c361eab91486261d48 Mon Sep 17 00:00:00 2001 From: chris-law-groundtruth Date: Mon, 21 Oct 2024 16:15:01 +0100 Subject: [PATCH 01/28] add financial services hook --- .../google/cloud/hooks/financial_services.py | 160 ++++++++++++++++++ 1 file changed, 160 insertions(+) create mode 100644 providers/src/airflow/providers/google/cloud/hooks/financial_services.py diff --git a/providers/src/airflow/providers/google/cloud/hooks/financial_services.py b/providers/src/airflow/providers/google/cloud/hooks/financial_services.py new file mode 100644 index 000000000000..891d4dea22ba --- /dev/null +++ b/providers/src/airflow/providers/google/cloud/hooks/financial_services.py @@ -0,0 +1,160 @@ +# 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 google.cloud import secretmanager +from googleapiclient.discovery import Resource, build + +from airflow.models import Variable +from airflow.providers.google.common.hooks.base_google import GoogleBaseHook + + +class FinancialServicesHook(GoogleBaseHook): + """ + Hook for interacting with the Google Financial Services API. + + Args: + location (str | None, optional): Location of the Google Cloud Platform + project. Defaults to None. + gcp_conn_id (str, optional): Identifier of connection to Google Cloud Platform. + Defaults to "google_cloud_default". + api_version (str, optional): API version for the Financial Services API. + Defaults to "v1". + dev_key_var (str, optional): Airflow variable name for accessing/saving the + developer key. If key is not provided, secret value will be stored in a + variable with the default name. Defaults to "AMLAI_API_KEY". + dev_key_secret_uri (str | None, optional): URI for the GCP secret (Secrets + Manager) containing the developer key. Secret will only be accessed if + dev_key_var does not exist. Defaults to None. + """ + + connection: build | None = None + + def __init__( + self, + location: str | None = None, + gcp_conn_id: str = "google_cloud_default", + api_version: str = "v1", + dev_key_var: str = "AMLAI_API_KEY", + dev_key_secret_uri: str | None = None, + **kwargs, + ) -> None: + if kwargs.get("delegate_to") is not None: + raise RuntimeError( + "The `delegate_to` parameter has been deprecated before and finally" + " removed in this version" + " of Google Provider. You MUST convert it to `impersonate_chain`" + ) + super().__init__( + gcp_conn_id=gcp_conn_id, + impersonation_chain=None, + ) + self.dev_key_var = dev_key_var + self.dev_key_secret_uri = dev_key_secret_uri + self.api_version = api_version + self.location = location + + def get_developer_key(self): + if Variable.get(key=self.dev_key_var, default_var=None): + developer_key = Variable.get(key=self.dev_key_var) + elif self.dev_key_secret_uri: + client = secretmanager.SecretManagerServiceClient() + secret = client.access_secret_version(name=self.dev_key_secret_uri) + developer_key = secret.payload.data.decode() + Variable.set(key=self.dev_key_var, value=developer_key) + else: + raise ValueError("Either dev_key_var or dev_key_secret_uri must be provided") + return developer_key + + def get_conn(self) -> Resource: + """ + Establish a connection to the Google Financial Services API. + + :return: a Google Cloud Financial Services service object. + """ + if not self.connection: + # TODO changes when allow listed + self.connection = build( + serviceName="financialservices", + version=self.api_version, + http=None, + cache_discovery=False, + discoveryServiceUrl="https://financialservices.googleapis.com/$discovery/rest?version=v1", + developerKey=self.get_developer_key(), + credentials=self.get_credentials(), + ) + + return self.connection + + def get_instance(self, instance_resource_uri: str) -> dict: + """ + Get an AML AI instance. + + Args: + instance_id (str): The instance identifier + project_id (str): Google Cloud Platform project_id containing the AML AI + instance resource + + Returns: + dict: A dictionary containing the instance metadata + """ + conn = self.get_conn() + response = conn.projects().locations().instances().get(name=instance_resource_uri).execute() + return response + + def create_instance(self, instance_id: str, kms_key: str, location_resource_uri: str) -> dict: + """ + Create an AML AI instance. + + Args: + instance_id (str): The ID of the instance + kms_key (str): The KMS key name used for CMEK (encryption-at-rest) + project_id (str): Google Cloud Platform project_id containing the AML AI + instance resource + + Returns: + dict: A dictionary containing metadata for the create instance + operation + """ + conn = self.get_conn() + response = ( + conn.projects() + .locations() + .instances() + .create( + parent=location_resource_uri, + instanceId=instance_id, + body={"kmsKey": kms_key}, + ) + .execute() + ) + return response + + def delete_instance(self, instance_resource_uri: str) -> dict: + """ + Delete an AML AI instance. + + Args: + instance_resource_uri (str): The ID of the instance + + Returns: + dict: A dictionary containing metadata for the delete instance + operation + """ + conn = self.get_conn() + response = conn.projects().locations().instances().delete(name=instance_resource_uri).execute() + return response From 8673f7f5065f5866a88cf5b7f20e5b60f2be2c14 Mon Sep 17 00:00:00 2001 From: chris-law-groundtruth Date: Mon, 21 Oct 2024 17:25:27 +0100 Subject: [PATCH 02/28] add create instance airflow operator --- .../google/cloud/hooks/financial_services.py | 8 +- .../cloud/operators/financial_services.py | 81 +++++++++++++++++++ 2 files changed, 83 insertions(+), 6 deletions(-) create mode 100644 providers/src/airflow/providers/google/cloud/operators/financial_services.py diff --git a/providers/src/airflow/providers/google/cloud/hooks/financial_services.py b/providers/src/airflow/providers/google/cloud/hooks/financial_services.py index 891d4dea22ba..8d9a20b7b9b7 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/financial_services.py +++ b/providers/src/airflow/providers/google/cloud/hooks/financial_services.py @@ -28,8 +28,6 @@ class FinancialServicesHook(GoogleBaseHook): Hook for interacting with the Google Financial Services API. Args: - location (str | None, optional): Location of the Google Cloud Platform - project. Defaults to None. gcp_conn_id (str, optional): Identifier of connection to Google Cloud Platform. Defaults to "google_cloud_default". api_version (str, optional): API version for the Financial Services API. @@ -46,7 +44,6 @@ class FinancialServicesHook(GoogleBaseHook): def __init__( self, - location: str | None = None, gcp_conn_id: str = "google_cloud_default", api_version: str = "v1", dev_key_var: str = "AMLAI_API_KEY", @@ -66,7 +63,6 @@ def __init__( self.dev_key_var = dev_key_var self.dev_key_secret_uri = dev_key_secret_uri self.api_version = api_version - self.location = location def get_developer_key(self): if Variable.get(key=self.dev_key_var, default_var=None): @@ -116,7 +112,7 @@ def get_instance(self, instance_resource_uri: str) -> dict: response = conn.projects().locations().instances().get(name=instance_resource_uri).execute() return response - def create_instance(self, instance_id: str, kms_key: str, location_resource_uri: str) -> dict: + def create_instance(self, instance_id: str, kms_key_uri: str, location_resource_uri: str) -> dict: """ Create an AML AI instance. @@ -138,7 +134,7 @@ def create_instance(self, instance_id: str, kms_key: str, location_resource_uri: .create( parent=location_resource_uri, instanceId=instance_id, - body={"kmsKey": kms_key}, + body={"kmsKey": kms_key_uri}, ) .execute() ) diff --git a/providers/src/airflow/providers/google/cloud/operators/financial_services.py b/providers/src/airflow/providers/google/cloud/operators/financial_services.py new file mode 100644 index 000000000000..035b3f885b89 --- /dev/null +++ b/providers/src/airflow/providers/google/cloud/operators/financial_services.py @@ -0,0 +1,81 @@ +# 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 typing import TYPE_CHECKING + +from airflow.providers.google.cloud.hooks.financial_services import FinancialServicesHook +from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator + +if TYPE_CHECKING: + from airflow.utils.context import Context + + +class FinancialServicesCreateInstanceOperator(GoogleCloudBaseOperator): + """ + Create a Financial Services AML AI Instance. + + Args: + instance_id (str): Identifier for the instance to create + location_resource_uri (str): URI of the location to create the instance in (format: 'projects//locations/) + kms_key_uri (str): URI of the KMS key to that will be used for instance encryption (format: 'projects//locations//keyRings//cryptoKeys/') + gcp_conn_id (str, optional): Identifier of connection to Google Cloud Platform. + Defaults to "google_cloud_default". + api_version (str, optional): API version for the Financial Services API. + Defaults to "v1". + dev_key_var (str, optional): Airflow variable name for accessing/saving the + developer key. If key is not provided, secret value will be stored in a + variable with the default name. Defaults to "AMLAI_API_KEY". + dev_key_secret_uri (str | None, optional): URI for the GCP secret (Secrets + Manager) containing the developer key. Secret will only be accessed if + dev_key_var does not exist. Defaults to None. + """ + + def __init__( + self, + instance_id: str, + location_resource_uri: str, + kms_key_uri: str, + gcp_conn_id: str = "google_cloud_default", + api_version: str = "v1", + dev_key_var: str = "AMLAI_API_KEY", + dev_key_secret_uri: str | None = None, + **kwargs, + ) -> None: + super().__init__(**kwargs) + self.instance_id = instance_id + self.location_resource_uri = location_resource_uri + self.gcp_conn_id = gcp_conn_id + self.api_version = api_version + self.dev_key_var = dev_key_var + self.dev_key_secret_uri = dev_key_secret_uri + self.kms_key_uri = kms_key_uri + + def execute(self, context: Context): + super().execute(context) + hook = FinancialServicesHook( + gcp_conn_id=self.gcp_conn_id, + api_version=self.api_version, + dev_key_var=self.dev_key_var, + dev_key_secret_uri=self.dev_key_secret_uri, + ) + response = hook.create_instance( + instance_id=self.instance_id, + kms_key_uri=self.kms_key_uri, + location_resource_uri=self.location_resource_uri, + ) + return response From e2a01962d0d964012532cfec39430876a02c26fa Mon Sep 17 00:00:00 2001 From: chris-law-groundtruth Date: Mon, 21 Oct 2024 17:57:31 +0100 Subject: [PATCH 03/28] add delete and get instance operators --- .../google/cloud/hooks/financial_services.py | 59 ++++---- .../cloud/operators/financial_services.py | 126 ++++++++++++++++-- 2 files changed, 141 insertions(+), 44 deletions(-) diff --git a/providers/src/airflow/providers/google/cloud/hooks/financial_services.py b/providers/src/airflow/providers/google/cloud/hooks/financial_services.py index 8d9a20b7b9b7..45adca0c34c5 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/financial_services.py +++ b/providers/src/airflow/providers/google/cloud/hooks/financial_services.py @@ -27,17 +27,16 @@ class FinancialServicesHook(GoogleBaseHook): """ Hook for interacting with the Google Financial Services API. - Args: - gcp_conn_id (str, optional): Identifier of connection to Google Cloud Platform. - Defaults to "google_cloud_default". - api_version (str, optional): API version for the Financial Services API. - Defaults to "v1". - dev_key_var (str, optional): Airflow variable name for accessing/saving the - developer key. If key is not provided, secret value will be stored in a - variable with the default name. Defaults to "AMLAI_API_KEY". - dev_key_secret_uri (str | None, optional): URI for the GCP secret (Secrets - Manager) containing the developer key. Secret will only be accessed if - dev_key_var does not exist. Defaults to None. + :param gcp_conn_id: Identifier of connection to Google Cloud Platform. + Defaults to "google_cloud_default". + :param api_version: API version for the Financial Services API. + Defaults to "v1". + :param dev_key_var: Airflow variable name for accessing/saving the + developer key. If key is not provided, secret value will be stored in a + variable with the default name. Defaults to "AMLAI_API_KEY". + :param dev_key_secret_uri: URI for the GCP secret (Secrets + Manager) containing the developer key. Secret will only be accessed if + dev_key_var does not exist. Defaults to None. """ connection: build | None = None @@ -64,7 +63,14 @@ def __init__( self.dev_key_secret_uri = dev_key_secret_uri self.api_version = api_version - def get_developer_key(self): + def get_developer_key(self) -> str: + """ + Get the developer API key for accessing the Financial Services discovery API. + + :raises ValueError: If dev_key_secret_uri = None and the developer key is not + saved in an Airflow variable named dev_key_var + :return str: API key for accessing the Financial Services discovery API + """ if Variable.get(key=self.dev_key_var, default_var=None): developer_key = Variable.get(key=self.dev_key_var) elif self.dev_key_secret_uri: @@ -100,13 +106,10 @@ def get_instance(self, instance_resource_uri: str) -> dict: """ Get an AML AI instance. - Args: - instance_id (str): The instance identifier - project_id (str): Google Cloud Platform project_id containing the AML AI - instance resource + :param instance_resource_uri: URI of the instance to get (format: + 'projects//locations//instances/) - Returns: - dict: A dictionary containing the instance metadata + :returns: A dictionary containing the instance metadata """ conn = self.get_conn() response = conn.projects().locations().instances().get(name=instance_resource_uri).execute() @@ -116,15 +119,12 @@ def create_instance(self, instance_id: str, kms_key_uri: str, location_resource_ """ Create an AML AI instance. - Args: - instance_id (str): The ID of the instance - kms_key (str): The KMS key name used for CMEK (encryption-at-rest) - project_id (str): Google Cloud Platform project_id containing the AML AI - instance resource + :param instance_id: Identifier for the instance to create + :param kms_key: URI of the KMS key to that will be used for instance encryption + (format: 'projects//locations//keyRings// + cryptoKeys/') - Returns: - dict: A dictionary containing metadata for the create instance - operation + :returns: A dictionary containing metadata for the create instance operation """ conn = self.get_conn() response = ( @@ -144,11 +144,10 @@ def delete_instance(self, instance_resource_uri: str) -> dict: """ Delete an AML AI instance. - Args: - instance_resource_uri (str): The ID of the instance + :param instance_resource_uri: URI of the instance to delete (format: + 'projects//locations//instances/) - Returns: - dict: A dictionary containing metadata for the delete instance + :returns: A dictionary containing metadata for the delete instance operation """ conn = self.get_conn() diff --git a/providers/src/airflow/providers/google/cloud/operators/financial_services.py b/providers/src/airflow/providers/google/cloud/operators/financial_services.py index 035b3f885b89..7d818d0dbe58 100644 --- a/providers/src/airflow/providers/google/cloud/operators/financial_services.py +++ b/providers/src/airflow/providers/google/cloud/operators/financial_services.py @@ -29,20 +29,22 @@ class FinancialServicesCreateInstanceOperator(GoogleCloudBaseOperator): """ Create a Financial Services AML AI Instance. - Args: - instance_id (str): Identifier for the instance to create - location_resource_uri (str): URI of the location to create the instance in (format: 'projects//locations/) - kms_key_uri (str): URI of the KMS key to that will be used for instance encryption (format: 'projects//locations//keyRings//cryptoKeys/') - gcp_conn_id (str, optional): Identifier of connection to Google Cloud Platform. - Defaults to "google_cloud_default". - api_version (str, optional): API version for the Financial Services API. - Defaults to "v1". - dev_key_var (str, optional): Airflow variable name for accessing/saving the - developer key. If key is not provided, secret value will be stored in a - variable with the default name. Defaults to "AMLAI_API_KEY". - dev_key_secret_uri (str | None, optional): URI for the GCP secret (Secrets - Manager) containing the developer key. Secret will only be accessed if - dev_key_var does not exist. Defaults to None. + :param instance_id: Identifier for the instance to create + :param location_resource_uri: URI of the location to create the instance in + (format: 'projects//locations/) + :param kms_key_uri: URI of the KMS key to that will be used for instance + encryption (format: 'projects//locations//keyRings/ + /cryptoKeys/') + :param gcp_conn_id: Identifier of connection to Google Cloud Platform. + Defaults to "google_cloud_default". + :param api_version: API version for the Financial Services API. + Defaults to "v1". + :param dev_key_var: Airflow variable name for accessing/saving the + developer key. If key is not provided, secret value will be stored in a + variable with the default name. Defaults to "AMLAI_API_KEY". + :param dev_key_secret_uri: URI for the GCP secret (Secrets + Manager) containing the developer key. Secret will only be accessed if + dev_key_var does not exist. Defaults to None. """ def __init__( @@ -79,3 +81,99 @@ def execute(self, context: Context): location_resource_uri=self.location_resource_uri, ) return response + + +class FinancialServicesDeleteInstanceOperator(GoogleCloudBaseOperator): + """ + Delete a Financial Services AML AI Instance. + + :param instance_resource_uri: URI of the instance to delete (format: + 'projects//locations//instances/) + :param gcp_conn_id: Identifier of connection to Google Cloud Platform. + Defaults to "google_cloud_default". + :param api_version: API version for the Financial Services API. + Defaults to "v1". + :param dev_key_var: Airflow variable name for accessing/saving the + developer key. If key is not provided, secret value will be stored in a + variable with the default name. Defaults to "AMLAI_API_KEY". + :param dev_key_secret_uri: URI for the GCP secret (Secrets + Manager) containing the developer key. Secret will only be accessed if + dev_key_var does not exist. Defaults to None. + """ + + def __init__( + self, + instance_resource_uri: str, + gcp_conn_id: str = "google_cloud_default", + api_version: str = "v1", + dev_key_var: str = "AMLAI_API_KEY", + dev_key_secret_uri: str | None = None, + **kwargs, + ) -> None: + super().__init__(**kwargs) + self.instance_resource_uri = instance_resource_uri + self.gcp_conn_id = gcp_conn_id + self.api_version = api_version + self.dev_key_var = dev_key_var + self.dev_key_secret_uri = dev_key_secret_uri + + def execute(self, context: Context): + super().execute(context) + hook = FinancialServicesHook( + gcp_conn_id=self.gcp_conn_id, + api_version=self.api_version, + dev_key_var=self.dev_key_var, + dev_key_secret_uri=self.dev_key_secret_uri, + ) + response = hook.delete_instance( + instance_resource_uri=self.instance_resource_uri, + ) + return response + + +class FinancialServicesGetInstanceOperator(GoogleCloudBaseOperator): + """ + Get a Financial Services AML AI Instance. + + :param instance_resource_uri: URI of the instance to get (format: + 'projects//locations//instances/) + :param gcp_conn_id: Identifier of connection to Google Cloud Platform. + Defaults to "google_cloud_default". + :param api_version: API version for the Financial Services API. + Defaults to "v1". + :param dev_key_var: Airflow variable name for accessing/saving the + developer key. If key is not provided, secret value will be stored in a + variable with the default name. Defaults to "AMLAI_API_KEY". + :param dev_key_secret_uri: URI for the GCP secret (Secrets + Manager) containing the developer key. Secret will only be accessed if + dev_key_var does not exist. Defaults to None. + """ + + def __init__( + self, + instance_resource_uri: str, + gcp_conn_id: str = "google_cloud_default", + api_version: str = "v1", + dev_key_var: str = "AMLAI_API_KEY", + dev_key_secret_uri: str | None = None, + **kwargs, + ) -> None: + super().__init__(**kwargs) + self.instance_resource_uri = instance_resource_uri + self.gcp_conn_id = gcp_conn_id + self.api_version = api_version + self.dev_key_var = dev_key_var + self.dev_key_secret_uri = dev_key_secret_uri + + def execute(self, context: Context): + super().execute(context) + hook = FinancialServicesHook( + gcp_conn_id=self.gcp_conn_id, + api_version=self.api_version, + dev_key_var=self.dev_key_var, + dev_key_secret_uri=self.dev_key_secret_uri, + ) + response = hook.get_instance( + instance_resource_uri=self.instance_resource_uri, + ) + return response From 44cd7a6d19624f11442638acea75ef36c36ce760 Mon Sep 17 00:00:00 2001 From: chris-law-groundtruth Date: Tue, 22 Oct 2024 12:11:33 +0100 Subject: [PATCH 04/28] add sensor for long running operations --- .../google/cloud/hooks/financial_services.py | 11 +++ .../cloud/sensors/financial_services.py | 76 +++++++++++++++++++ 2 files changed, 87 insertions(+) create mode 100644 providers/src/airflow/providers/google/cloud/sensors/financial_services.py diff --git a/providers/src/airflow/providers/google/cloud/hooks/financial_services.py b/providers/src/airflow/providers/google/cloud/hooks/financial_services.py index 45adca0c34c5..dc4df6812166 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/financial_services.py +++ b/providers/src/airflow/providers/google/cloud/hooks/financial_services.py @@ -153,3 +153,14 @@ def delete_instance(self, instance_resource_uri: str) -> dict: conn = self.get_conn() response = conn.projects().locations().instances().delete(name=instance_resource_uri).execute() return response + + def get_operation(self, operation_resource_uri: str) -> dict: + """ + Get an operation. + + :param operation_resource_uri: URI of the operation + :return: A dictionary containing metadata for the operation + """ + conn = self.get_conn() + response = conn.projects().locations().operations().get(name=operation_resource_uri).execute() + return response diff --git a/providers/src/airflow/providers/google/cloud/sensors/financial_services.py b/providers/src/airflow/providers/google/cloud/sensors/financial_services.py new file mode 100644 index 000000000000..5f5838af029e --- /dev/null +++ b/providers/src/airflow/providers/google/cloud/sensors/financial_services.py @@ -0,0 +1,76 @@ +# 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 typing import TYPE_CHECKING, Sequence + +from airflow.exceptions import AirflowFailException +from airflow.providers.google.cloud.hooks.financial_services import FinancialServicesHook +from airflow.sensors.base import BaseSensorOperator, PokeReturnValue + +if TYPE_CHECKING: + from airflow.utils.context import Context + + +class FinancialServicesOperationSensor(BaseSensorOperator): + """ + Check status of AML AI operation. + + :param operation_resource_uri: URI of the operation (format: + 'projects//locations//operations/) + :param gcp_conn_id: Identifier of connection to Google Cloud Platform. + Defaults to "google_cloud_default". + :param api_version: API version for the Financial Services API. + Defaults to "v1". + :param dev_key_var: Airflow variable name for accessing/saving the + developer key. If key is not provided, secret value will be stored in a + variable with the default name. Defaults to "AMLAI_API_KEY". + :param dev_key_secret_uri: URI for the GCP secret (Secrets + Manager) containing the developer key. Secret will only be accessed if + dev_key_var does not exist. Defaults to None. + """ + + template_fields: Sequence[str] = ("operation_resource_uri",) + + def __init__( + self, + operation_resource_uri: str, + gcp_conn_id: str = "google_cloud_default", + api_version: str = "v1", + dev_key_var: str = "AMLAI_API_KEY", + dev_key_secret_uri: str | None = None, + **kwargs, + ): + super().__init__(**kwargs) + self.operation_resource_uri = operation_resource_uri + self.gcp_conn_id = gcp_conn_id + self.api_version = api_version + self.dev_key_var = dev_key_var + self.dev_key_secret_uri = dev_key_secret_uri + + def poke(self, context: Context) -> PokeReturnValue: + super().poke(context) + hook = FinancialServicesHook( + gcp_conn_id=self.gcp_conn_id, + api_version=self.api_version, + dev_key_var=self.dev_key_var, + dev_key_secret_uri=self.dev_key_secret_uri, + ) + operation = hook.get_operation(operation_resource_uri=self.operation_resource_uri) + if "error" in operation.keys(): + raise AirflowFailException(operation["error"]) + return PokeReturnValue(is_done=operation["done"]) From 3477b2502ea378d6f22a3af0951ae928bccefbc1 Mon Sep 17 00:00:00 2001 From: chris-law-groundtruth Date: Tue, 22 Oct 2024 12:22:06 +0100 Subject: [PATCH 05/28] allow templated fields in financial services operators --- .../google/cloud/operators/financial_services.py | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/providers/src/airflow/providers/google/cloud/operators/financial_services.py b/providers/src/airflow/providers/google/cloud/operators/financial_services.py index 7d818d0dbe58..bb06c0a484ca 100644 --- a/providers/src/airflow/providers/google/cloud/operators/financial_services.py +++ b/providers/src/airflow/providers/google/cloud/operators/financial_services.py @@ -16,7 +16,7 @@ # under the License. from __future__ import annotations -from typing import TYPE_CHECKING +from typing import TYPE_CHECKING, Sequence from airflow.providers.google.cloud.hooks.financial_services import FinancialServicesHook from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator @@ -47,6 +47,8 @@ class FinancialServicesCreateInstanceOperator(GoogleCloudBaseOperator): dev_key_var does not exist. Defaults to None. """ + template_fields: Sequence[str] = ("instance_id", "location_resource_uri", "kms_key_uri") + def __init__( self, instance_id: str, @@ -101,6 +103,8 @@ class FinancialServicesDeleteInstanceOperator(GoogleCloudBaseOperator): dev_key_var does not exist. Defaults to None. """ + template_fields: Sequence[str] = "instance_resource_uri" + def __init__( self, instance_resource_uri: str, @@ -149,6 +153,8 @@ class FinancialServicesGetInstanceOperator(GoogleCloudBaseOperator): dev_key_var does not exist. Defaults to None. """ + template_fields: Sequence[str] = "instance_resource_uri" + def __init__( self, instance_resource_uri: str, From 3f5abef656b5e0a2edf5f8402dc929d37dd6ebf4 Mon Sep 17 00:00:00 2001 From: chris-law-groundtruth Date: Tue, 22 Oct 2024 14:08:36 +0100 Subject: [PATCH 06/28] add example DAG for financial services --- .../example_financial_services.py | 79 +++++++++++++++++++ .../cloud/operators/financial_services.py | 4 +- 2 files changed, 81 insertions(+), 2 deletions(-) create mode 100644 providers/src/airflow/providers/google/cloud/example_dags/example_financial_services.py diff --git a/providers/src/airflow/providers/google/cloud/example_dags/example_financial_services.py b/providers/src/airflow/providers/google/cloud/example_dags/example_financial_services.py new file mode 100644 index 000000000000..8618ab346d84 --- /dev/null +++ b/providers/src/airflow/providers/google/cloud/example_dags/example_financial_services.py @@ -0,0 +1,79 @@ +# 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 uuid +from datetime import datetime, timedelta + +from airflow.models.dag import DAG +from airflow.operators.python import PythonOperator +from airflow.providers.google.cloud.operators.financial_services import ( + FinancialServicesCreateInstanceOperator, + FinancialServicesGetInstanceOperator, +) +from airflow.providers.google.cloud.sensors.financial_services import FinancialServicesOperationSensor + +params = { + "project_id": None, + "location": None, + "keyRing": None, + "cryptoKey": None, +} + +with DAG( + "example_financial_services", + start_date=datetime(2024, 1, 1), + catchup=False, +) as dag: + # TODO requires an airflow variable containing the developer key / developer key secret resource URI + + generate_run_id_task = PythonOperator( + task_id="generate_run_id_task", python_callable=uuid.uuid4().__str__ + ) + + create_instance_task = FinancialServicesCreateInstanceOperator( + task_id="create_instance_task", + instance_id="instance_{{ task_instance.xcom_pull(task_ids='generate_run_id_task', key='return_value') }}", + location_resource_uri="projects/{{ params.project_id }}/locations/{{ params.location }}", + kms_key_uri="projects/{{ params.project_id }}/locations/{{ params.location }}/keyRings/{{ params.keyRing }}/cryptoKeys/{{ params.cryptoKey }}", + ) + + create_instance_sensor = FinancialServicesOperationSensor( + task_id="create_instance_sensor", + operation_resource_uri="{{ task_instance.xcom_pull(task_ids='create_instance_task', key='return_value') }}", + poke_interval=timedelta(minutes=1), + timeout=timedelta(days=1), + ) + + get_instance_task = FinancialServicesGetInstanceOperator( + task_id="get_instance_task", + instance_resource_uri="projects/{{ params.project_id }}/locations/{{ params.location }}/instances/instance_{{ task_instance.xcom_pull(task_ids='generate_run_id_task', key='return_value') }}", + ) + + # TODO Log the task info + + delete_instance_task = FinancialServicesGetInstanceOperator( + task_id="delete_instance_task", + instance_resource_uri="projects/{{ params.project_id }}/locations/{{ params.location }}/instances/instance_{{ task_instance.xcom_pull(task_ids='generate_run_id_task', key='return_value') }}", + ) + + delete_instance_sensor = FinancialServicesOperationSensor( + task_id="delete_instance_sensor", + operation_resource_uri="{{ task_instance.xcom_pull(task_ids='delete_instance_task', key='return_value') }}", + poke_interval=timedelta(minutes=1), + timeout=timedelta(days=1), + ) diff --git a/providers/src/airflow/providers/google/cloud/operators/financial_services.py b/providers/src/airflow/providers/google/cloud/operators/financial_services.py index bb06c0a484ca..cd04eb904d13 100644 --- a/providers/src/airflow/providers/google/cloud/operators/financial_services.py +++ b/providers/src/airflow/providers/google/cloud/operators/financial_services.py @@ -82,7 +82,7 @@ def execute(self, context: Context): kms_key_uri=self.kms_key_uri, location_resource_uri=self.location_resource_uri, ) - return response + return response["name"] class FinancialServicesDeleteInstanceOperator(GoogleCloudBaseOperator): @@ -132,7 +132,7 @@ def execute(self, context: Context): response = hook.delete_instance( instance_resource_uri=self.instance_resource_uri, ) - return response + return response["name"] class FinancialServicesGetInstanceOperator(GoogleCloudBaseOperator): From ed45f0348fb6d8fd0783b97c8a7516d256a4587b Mon Sep 17 00:00:00 2001 From: chris-law-groundtruth Date: Wed, 23 Oct 2024 11:53:55 +0100 Subject: [PATCH 07/28] add hook unit tests for financial services financial_services_discovery.json is added because the discovery doc requires a developer key when accessed through the googleapiclient.discovery.build method --- .../google/cloud/hooks/financial_services.py | 41 +- .../hooks/financial_services_discovery.json | 2746 +++++++++++++++++ .../cloud/hooks/test_financial_services.py | 120 + 3 files changed, 2875 insertions(+), 32 deletions(-) create mode 100644 providers/src/airflow/providers/google/cloud/hooks/financial_services_discovery.json create mode 100644 providers/tests/google/cloud/hooks/test_financial_services.py diff --git a/providers/src/airflow/providers/google/cloud/hooks/financial_services.py b/providers/src/airflow/providers/google/cloud/hooks/financial_services.py index dc4df6812166..8680c302ed8e 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/financial_services.py +++ b/providers/src/airflow/providers/google/cloud/hooks/financial_services.py @@ -16,10 +16,11 @@ # under the License. from __future__ import annotations -from google.cloud import secretmanager -from googleapiclient.discovery import Resource, build +import importlib.resources +import json + +from googleapiclient.discovery import Resource, build_from_document -from airflow.models import Variable from airflow.providers.google.common.hooks.base_google import GoogleBaseHook @@ -39,7 +40,7 @@ class FinancialServicesHook(GoogleBaseHook): dev_key_var does not exist. Defaults to None. """ - connection: build | None = None + connection: Resource | None = None def __init__( self, @@ -63,25 +64,6 @@ def __init__( self.dev_key_secret_uri = dev_key_secret_uri self.api_version = api_version - def get_developer_key(self) -> str: - """ - Get the developer API key for accessing the Financial Services discovery API. - - :raises ValueError: If dev_key_secret_uri = None and the developer key is not - saved in an Airflow variable named dev_key_var - :return str: API key for accessing the Financial Services discovery API - """ - if Variable.get(key=self.dev_key_var, default_var=None): - developer_key = Variable.get(key=self.dev_key_var) - elif self.dev_key_secret_uri: - client = secretmanager.SecretManagerServiceClient() - secret = client.access_secret_version(name=self.dev_key_secret_uri) - developer_key = secret.payload.data.decode() - Variable.set(key=self.dev_key_var, value=developer_key) - else: - raise ValueError("Either dev_key_var or dev_key_secret_uri must be provided") - return developer_key - def get_conn(self) -> Resource: """ Establish a connection to the Google Financial Services API. @@ -89,16 +71,11 @@ def get_conn(self) -> Resource: :return: a Google Cloud Financial Services service object. """ if not self.connection: - # TODO changes when allow listed - self.connection = build( - serviceName="financialservices", - version=self.api_version, - http=None, - cache_discovery=False, - discoveryServiceUrl="https://financialservices.googleapis.com/$discovery/rest?version=v1", - developerKey=self.get_developer_key(), - credentials=self.get_credentials(), + api_doc_res = importlib.resources.files("airflow.providers.google.cloud.hooks").joinpath( + "financial_services_discovery.json" ) + with importlib.resources.as_file(api_doc_res) as file: + self.connection = build_from_document(json.loads(file.read_text())) return self.connection diff --git a/providers/src/airflow/providers/google/cloud/hooks/financial_services_discovery.json b/providers/src/airflow/providers/google/cloud/hooks/financial_services_discovery.json new file mode 100644 index 000000000000..2a10a6ea8c17 --- /dev/null +++ b/providers/src/airflow/providers/google/cloud/hooks/financial_services_discovery.json @@ -0,0 +1,2746 @@ +{ + "schemas": { + "ListOperationsResponse": { + "id": "ListOperationsResponse", + "description": "The response message for Operations.ListOperations.", + "type": "object", + "properties": { + "operations": { + "description": "A list of operations that matches the specified filter in the request.", + "type": "array", + "items": { + "$ref": "Operation" + } + }, + "nextPageToken": { + "description": "The standard List next-page token.", + "type": "string" + } + } + }, + "Operation": { + "id": "Operation", + "description": "This resource represents a long-running operation that is the result of a network API call.", + "type": "object", + "properties": { + "name": { + "description": "The server-assigned name, which is only unique within the same service that originally returns it. If you use the default HTTP mapping, the `name` should be a resource name ending with `operations/{unique_id}`.", + "type": "string" + }, + "metadata": { + "description": "Service-specific metadata associated with the operation. It typically contains progress information and common metadata such as create time. Some services might not provide such metadata. Any method that returns a long-running operation should document the metadata type, if any.", + "type": "object", + "additionalProperties": { + "type": "any", + "description": "Properties of the object. Contains field @type with type URL." + } + }, + "done": { + "description": "If the value is `false`, it means the operation is still in progress. If `true`, the operation is completed, and either `error` or `response` is available.", + "type": "boolean" + }, + "error": { + "description": "The error result of the operation in case of failure or cancellation.", + "$ref": "Status" + }, + "response": { + "description": "The normal, successful response of the operation. If the original method returns no data on success, such as `Delete`, the response is `google.protobuf.Empty`. If the original method is standard `Get`/`Create`/`Update`, the response should be the resource. For other methods, the response should have the type `XxxResponse`, where `Xxx` is the original method name. For example, if the original method name is `TakeSnapshot()`, the inferred response type is `TakeSnapshotResponse`.", + "type": "object", + "additionalProperties": { + "type": "any", + "description": "Properties of the object. Contains field @type with type URL." + } + } + } + }, + "Status": { + "id": "Status", + "description": "The `Status` type defines a logical error model that is suitable for different programming environments, including REST APIs and RPC APIs. It is used by [gRPC](https://github.com/grpc). Each `Status` message contains three pieces of data: error code, error message, and error details. You can find out more about this error model and how to work with it in the [API Design Guide](https://cloud.google.com/apis/design/errors).", + "type": "object", + "properties": { + "code": { + "description": "The status code, which should be an enum value of google.rpc.Code.", + "type": "integer", + "format": "int32" + }, + "message": { + "description": "A developer-facing error message, which should be in English. Any user-facing error message should be localized and sent in the google.rpc.Status.details field, or localized by the client.", + "type": "string" + }, + "details": { + "description": "A list of messages that carry the error details. There is a common set of message types for APIs to use.", + "type": "array", + "items": { + "type": "object", + "additionalProperties": { + "type": "any", + "description": "Properties of the object. Contains field @type with type URL." + } + } + } + } + }, + "Empty": { + "id": "Empty", + "description": "A generic empty message that you can reuse to avoid defining duplicated empty messages in your APIs. A typical example is to use it as the request or the response type of an API method. For instance: service Foo { rpc Bar(google.protobuf.Empty) returns (google.protobuf.Empty); }", + "type": "object", + "properties": {} + }, + "CancelOperationRequest": { + "id": "CancelOperationRequest", + "description": "The request message for Operations.CancelOperation.", + "type": "object", + "properties": {} + }, + "GoogleCloudFinancialservicesV1ListInstancesResponse": { + "id": "GoogleCloudFinancialservicesV1ListInstancesResponse", + "description": "Response for retrieving a list of Instances", + "type": "object", + "properties": { + "instances": { + "description": "List of Instance resources", + "type": "array", + "items": { + "$ref": "GoogleCloudFinancialservicesV1Instance" + } + }, + "nextPageToken": { + "description": "This token should be passed to the next ListInstancesRequest to retrieve the next page of Instances.", + "type": "string" + }, + "unreachable": { + "description": "Locations that could not be reached.", + "type": "array", + "items": { + "type": "string" + } + } + } + }, + "GoogleCloudFinancialservicesV1Instance": { + "id": "GoogleCloudFinancialservicesV1Instance", + "description": "Instance is a container for the rest of API resources. Only resources in the same instance can interact with each other. Child resources inherit the location (data residency) and encryption (CMEK). The location of the provided input and output in requests must match the location of the instance.", + "type": "object", + "properties": { + "name": { + "description": "Output only. The full path to the Instance resource in this API. format: \"projects/{project}/locations/{location}/instances/{instance}\"", + "readOnly": true, + "type": "string" + }, + "createTime": { + "description": "Output only. Timestamp when the Instance was created. Assigned by the server.", + "readOnly": true, + "type": "string", + "format": "google-datetime" + }, + "updateTime": { + "description": "Output only. Timestamp when the Instance was last updated. Assigned by the server.", + "readOnly": true, + "type": "string", + "format": "google-datetime" + }, + "state": { + "description": "Output only. State of the instance. Assigned by the server.", + "readOnly": true, + "type": "string", + "enumDescriptions": [ + "State is unspecified, should not occur.", + "The resource has not finished being created.", + "The resource is active/ready to be used.", + "The resource is in the process of being updated.", + "The resource is in the process of being deleted." + ], + "enum": [ + "STATE_UNSPECIFIED", + "CREATING", + "ACTIVE", + "UPDATING", + "DELETING" + ] + }, + "labels": { + "description": "Labels", + "type": "object", + "additionalProperties": { + "type": "string" + } + }, + "kmsKey": { + "description": "Required. The KMS key name used for CMEK (encryption-at-rest). format: \"projects/{project}/locations/{location}/keyRings/{keyRing}/cryptoKeys/{cryptoKey}\" VPC-SC restrictions apply.", + "type": "string" + } + } + }, + "GoogleCloudFinancialservicesV1ImportRegisteredPartiesRequest": { + "id": "GoogleCloudFinancialservicesV1ImportRegisteredPartiesRequest", + "description": "Request for adding/removing registered parties from BigQuery tables specified by the customer.", + "type": "object", + "properties": { + "partyTables": { + "description": "Optional. List of BigQuery tables. Union of tables will be taken if there is more than one table. VPC-SC restrictions apply. format: \"bq://{project}.{bqDatasetID}.{bqTableID}\" Use of `datasets` is preferred over the latter due to its simplicity and the reduced risk of errors `party_tables` and `datasets` must not be provided at the same time", + "type": "array", + "items": { + "type": "string" + } + }, + "mode": { + "description": "Required. Mode of the request.", + "type": "string", + "enumDescriptions": [ + "Default mode.", + "Replace parties that are removable in Parties Table with new parties.", + "Add new parties to Parties Table." + ], + "enum": [ + "UPDATE_MODE_UNSPECIFIED", + "REPLACE", + "APPEND" + ] + }, + "validateOnly": { + "description": "Optional. Is the request will not register the parties, just determine what would happen.", + "type": "boolean" + }, + "lineOfBusiness": { + "description": "Required. LineOfBusiness for the specified registered parties.", + "type": "string", + "enumDescriptions": [ + "An unspecified LineOfBusiness. Do not use.", + "Commercial LineOfBusiness.", + "Retail LineOfBusiness." + ], + "enum": [ + "LINE_OF_BUSINESS_UNSPECIFIED", + "COMMERCIAL", + "RETAIL" + ] + } + } + }, + "GoogleCloudFinancialservicesV1ExportRegisteredPartiesRequest": { + "id": "GoogleCloudFinancialservicesV1ExportRegisteredPartiesRequest", + "description": "Request to export a list of currently registered parties.", + "type": "object", + "properties": { + "dataset": { + "description": "Required. The location to output the RegisteredParties.", + "$ref": "GoogleCloudFinancialservicesV1BigQueryDestination" + }, + "lineOfBusiness": { + "description": "Required. LineOfBusiness to get RegisteredParties from.", + "type": "string", + "enumDescriptions": [ + "An unspecified LineOfBusiness. Do not use.", + "Commercial LineOfBusiness.", + "Retail LineOfBusiness." + ], + "enum": [ + "LINE_OF_BUSINESS_UNSPECIFIED", + "COMMERCIAL", + "RETAIL" + ] + } + } + }, + "GoogleCloudFinancialservicesV1BigQueryDestination": { + "id": "GoogleCloudFinancialservicesV1BigQueryDestination", + "description": "BigQueryDestination is a wrapper for BigQuery output information.", + "type": "object", + "properties": { + "tableUri": { + "description": "Optional. BigQuery URI to a table, must be of the form bq://projectId.bqDatasetId.tableId. Note that the BigQuery dataset must already exist. VPC-SC restrictions apply.", + "type": "string" + }, + "writeDisposition": { + "description": "Required. Whether to overwrite or not destination table. By default the table won't be overwritten and an error will be returned if the table exists and contains data.", + "type": "string", + "enumDescriptions": [ + "Default behavior is the same as WRITE_EMPTY.", + "If the table already exists and contains data, an error is returned.", + "If the table already exists, the data will be overwritten." + ], + "enum": [ + "WRITE_DISPOSITION_UNSPECIFIED", + "WRITE_EMPTY", + "WRITE_TRUNCATE" + ] + } + } + }, + "GoogleCloudFinancialservicesV1ListDatasetsResponse": { + "id": "GoogleCloudFinancialservicesV1ListDatasetsResponse", + "description": "Response for retrieving a list of Datasets", + "type": "object", + "properties": { + "datasets": { + "description": "List of Dataset resources", + "type": "array", + "items": { + "$ref": "GoogleCloudFinancialservicesV1Dataset" + } + }, + "nextPageToken": { + "description": "This token should be passed to the next ListDatasetsRequest to retrieve the next page of Datasets (empty indicicates we aer done).", + "type": "string" + }, + "unreachable": { + "description": "Locations that could not be reached.", + "type": "array", + "items": { + "type": "string" + } + } + } + }, + "GoogleCloudFinancialservicesV1Dataset": { + "id": "GoogleCloudFinancialservicesV1Dataset", + "description": "The Dataset resource contains summary information about a dataset.", + "type": "object", + "properties": { + "name": { + "description": "Output only. The resource name of the Dataset. format: \"/projects/{project_num}/locations/{location}/instances/{instance}/datasets/{dataset}\"", + "readOnly": true, + "type": "string" + }, + "createTime": { + "description": "Output only. The timestamp of creation of this resource.", + "readOnly": true, + "type": "string", + "format": "google-datetime" + }, + "updateTime": { + "description": "Output only. The timestamp of the most recent update of this resource.", + "readOnly": true, + "type": "string", + "format": "google-datetime" + }, + "labels": { + "description": "Labels", + "type": "object", + "additionalProperties": { + "type": "string" + } + }, + "tableSpecs": { + "description": "Required. The set of BigQuery tables in the dataset. The key should be the table type and the value should be the BigQuery tables in the format `bq://{project}.{dataset}.{table}`. Current table types are: * `party` * `account_party_link` * `transaction` * `risk_case_event` * `party_supplementary_data`", + "type": "object", + "additionalProperties": { + "type": "string" + } + }, + "state": { + "description": "Output only. State of the dataset (creating, active, deleting, etc.)", + "readOnly": true, + "type": "string", + "enumDescriptions": [ + "State is unspecified, should not occur.", + "The resource has not finished being created.", + "The resource is active/ready to be used.", + "The resource is in the process of being updated.", + "The resource is in the process of being deleted." + ], + "enum": [ + "STATE_UNSPECIFIED", + "CREATING", + "ACTIVE", + "UPDATING", + "DELETING" + ] + }, + "dateRange": { + "description": "Required. Core time window of the dataset. All tables should have complete data covering this period.", + "$ref": "Interval" + }, + "timeZone": { + "description": "The timezone of the data, default will act as UTC.", + "$ref": "TimeZone" + } + } + }, + "Interval": { + "id": "Interval", + "description": "Represents a time interval, encoded as a Timestamp start (inclusive) and a Timestamp end (exclusive). The start must be less than or equal to the end. When the start equals the end, the interval is empty (matches no time). When both start and end are unspecified, the interval matches any time.", + "type": "object", + "properties": { + "startTime": { + "description": "Optional. Inclusive start of the interval. If specified, a Timestamp matching this interval will have to be the same or after the start.", + "type": "string", + "format": "google-datetime" + }, + "endTime": { + "description": "Optional. Exclusive end of the interval. If specified, a Timestamp matching this interval will have to be before the end.", + "type": "string", + "format": "google-datetime" + } + } + }, + "TimeZone": { + "id": "TimeZone", + "description": "Represents a time zone from the [IANA Time Zone Database](https://www.iana.org/time-zones).", + "type": "object", + "properties": { + "id": { + "description": "IANA Time Zone Database time zone, e.g. \"America/New_York\".", + "type": "string" + }, + "version": { + "description": "Optional. IANA Time Zone Database version number, e.g. \"2019a\".", + "type": "string" + } + } + }, + "GoogleCloudFinancialservicesV1ListModelsResponse": { + "id": "GoogleCloudFinancialservicesV1ListModelsResponse", + "description": "Response for retrieving a list of Models", + "type": "object", + "properties": { + "models": { + "description": "List of Model resources", + "type": "array", + "items": { + "$ref": "GoogleCloudFinancialservicesV1Model" + } + }, + "nextPageToken": { + "description": "This token should be passed to the next ListModelsRequest to retrieve the next page of Models (empty indicicates we are done).", + "type": "string" + }, + "unreachable": { + "description": "Locations that could not be reached.", + "type": "array", + "items": { + "type": "string" + } + } + } + }, + "GoogleCloudFinancialservicesV1Model": { + "id": "GoogleCloudFinancialservicesV1Model", + "description": "Model represents a trained model.", + "type": "object", + "properties": { + "name": { + "description": "Output only. The resource name of the Model. format: \"/projects/{project_num}/locations/{location}/instances/{instance}/models/{model}\"", + "readOnly": true, + "type": "string" + }, + "createTime": { + "description": "Output only. The timestamp of creation of this resource.", + "readOnly": true, + "type": "string", + "format": "google-datetime" + }, + "updateTime": { + "description": "Output only. The timestamp of the most recent update of this resource.", + "readOnly": true, + "type": "string", + "format": "google-datetime" + }, + "labels": { + "description": "Labels", + "type": "object", + "additionalProperties": { + "type": "string" + } + }, + "state": { + "description": "Output only. State of the model (creating, active, deleting, etc.)", + "readOnly": true, + "type": "string", + "enumDescriptions": [ + "State is unspecified, should not occur.", + "The resource has not finished being created.", + "The resource is active/ready to be used.", + "The resource is in the process of being updated.", + "The resource is in the process of being deleted." + ], + "enum": [ + "STATE_UNSPECIFIED", + "CREATING", + "ACTIVE", + "UPDATING", + "DELETING" + ] + }, + "engineVersion": { + "description": "Output only. The EngineVersion used in training this model. This is output only, and is determined from the EngineConfig used.", + "readOnly": true, + "type": "string" + }, + "engineConfig": { + "description": "Required. The resource name of the EngineConfig the model training will be based on. Format: \"/projects/{project_num}/locations/{location}/instances/{instance}/engineConfigs/{engineConfig}\"", + "type": "string" + }, + "primaryDataset": { + "description": "Required. The resource name of the Primary Dataset used in this model training. For information about how primary and auxiliary datasets are used, refer to the engine version's documentation. Format: \"/projects/{project_num}/locations/{location}/instances/{instance}/datasets/{dataset}\"", + "type": "string" + }, + "endTime": { + "description": "Required. End_time specifies the latest time from which labels are used and from which data is used to generate features for training. End_time should be no later than the end of the date_range of the dataset.", + "type": "string", + "format": "google-datetime" + }, + "lineOfBusiness": { + "description": "Output only. The line of business (Retail/Commercial) this model is used for. Determined by EngineConfig, cannot be set by user.", + "readOnly": true, + "type": "string", + "enumDescriptions": [ + "An unspecified LineOfBusiness. Do not use.", + "Commercial LineOfBusiness.", + "Retail LineOfBusiness." + ], + "enum": [ + "LINE_OF_BUSINESS_UNSPECIFIED", + "COMMERCIAL", + "RETAIL" + ] + } + } + }, + "GoogleCloudFinancialservicesV1ExportModelMetadataRequest": { + "id": "GoogleCloudFinancialservicesV1ExportModelMetadataRequest", + "description": "Request for exporting Model metadata.", + "type": "object", + "properties": { + "structuredMetadataDestination": { + "description": "Required. BigQuery output where the metadata will be written.", + "$ref": "GoogleCloudFinancialservicesV1BigQueryDestination" + } + } + }, + "GoogleCloudFinancialservicesV1ListEngineConfigsResponse": { + "id": "GoogleCloudFinancialservicesV1ListEngineConfigsResponse", + "description": "Response for retrieving a list of EngineConfigs", + "type": "object", + "properties": { + "engineConfigs": { + "description": "List of EngineConfig resources", + "type": "array", + "items": { + "$ref": "GoogleCloudFinancialservicesV1EngineConfig" + } + }, + "nextPageToken": { + "description": "This token should be passed to the next ListEngineConfigsRequest to retrieve the next page of EngineConfigs (empty indicicates we aer done).", + "type": "string" + }, + "unreachable": { + "description": "Locations that could not be reached.", + "type": "array", + "items": { + "type": "string" + } + } + } + }, + "GoogleCloudFinancialservicesV1EngineConfig": { + "id": "GoogleCloudFinancialservicesV1EngineConfig", + "description": "The EngineConfig resource creates the configuration for training a model.", + "type": "object", + "properties": { + "name": { + "description": "Output only. The resource name of the EngineConfig. format: \"/projects/{project_num}/locations/{location}/instances/{instance}/engineConfigs/{engine_config}\"", + "readOnly": true, + "type": "string" + }, + "createTime": { + "description": "Output only. The timestamp of creation of this resource.", + "readOnly": true, + "type": "string", + "format": "google-datetime" + }, + "updateTime": { + "description": "Output only. The timestamp of the most recent update of this resource.", + "readOnly": true, + "type": "string", + "format": "google-datetime" + }, + "labels": { + "description": "Labels", + "type": "object", + "additionalProperties": { + "type": "string" + } + }, + "state": { + "description": "Output only. State of the EngineConfig (creating, active, deleting, etc.)", + "readOnly": true, + "type": "string", + "enumDescriptions": [ + "State is unspecified, should not occur.", + "The resource has not finished being created.", + "The resource is active/ready to be used.", + "The resource is in the process of being updated.", + "The resource is in the process of being deleted." + ], + "enum": [ + "STATE_UNSPECIFIED", + "CREATING", + "ACTIVE", + "UPDATING", + "DELETING" + ] + }, + "engineVersion": { + "description": "Required. The resource name of the EngineVersion used in this model tuning. format: \"/projects/{project_num}/locations/{location}/instances/{instance}/engineVersions/{engine_version}\"", + "type": "string" + }, + "tuning": { + "description": "Optional. Configuration for tuning in creation of the EngineConfig. This field is required if `hyperparameter_source.type` is not `INHERITED`, and output-only otherwise.", + "$ref": "GoogleCloudFinancialservicesV1EngineConfigTuning" + }, + "performanceTarget": { + "description": "Optional. PerformanceTarget gives information on how the tuning and training will be evaluated. This field is required if `hyperparameter_source.type` is not `INHERITED`, and output-only otherwise.", + "$ref": "GoogleCloudFinancialservicesV1EngineConfigPerformanceTarget" + }, + "lineOfBusiness": { + "description": "Output only. The line of business (Retail/Commercial) this engine config is used for. Determined by EngineVersion, cannot be set by user.", + "readOnly": true, + "type": "string", + "enumDescriptions": [ + "An unspecified LineOfBusiness. Do not use.", + "Commercial LineOfBusiness.", + "Retail LineOfBusiness." + ], + "enum": [ + "LINE_OF_BUSINESS_UNSPECIFIED", + "COMMERCIAL", + "RETAIL" + ] + }, + "hyperparameterSourceType": { + "description": "Optional. The origin of hyperparameters for the created EngineConfig. The default is `TUNING`. In this case, the hyperparameters are selected as a result of a tuning run.", + "type": "string", + "enumDescriptions": [ + "Hyperparameter source type is unspecified, defaults to TUNING.", + "The EngineConfig creation starts a tuning job which selects the best hyperparameters.", + "The hyperparameters are inherited from another EngineConfig." + ], + "enum": [ + "HYPERPARAMETER_SOURCE_TYPE_UNSPECIFIED", + "TUNING", + "INHERITED" + ] + }, + "hyperparameterSource": { + "description": "Optional. Configuration of hyperparameters source EngineConfig.", + "$ref": "GoogleCloudFinancialservicesV1EngineConfigHyperparameterSource" + } + } + }, + "GoogleCloudFinancialservicesV1EngineConfigTuning": { + "id": "GoogleCloudFinancialservicesV1EngineConfigTuning", + "description": "The parameters needed for the tuning operation, these are used only in tuning and not passed on to training.", + "type": "object", + "properties": { + "primaryDataset": { + "description": "Required. The resource name of the Primary Dataset used in this model tuning. For information about how primary and auxiliary datasets are used, refer to the engine version's documentation. Format: \"/projects/{project_num}/locations/{location}/instances/{instance}/datasets/{dataset}\"", + "type": "string" + }, + "endTime": { + "description": "Required. End_time specifies the latest time from which labels are used and from which data is used to generate features for tuning. End_time should be no later than the end of the date_range of the dataset.", + "type": "string", + "format": "google-datetime" + } + } + }, + "GoogleCloudFinancialservicesV1EngineConfigPerformanceTarget": { + "id": "GoogleCloudFinancialservicesV1EngineConfigPerformanceTarget", + "description": "PerformanceTarget gives hints on how to evaluate the performance of a model.", + "type": "object", + "properties": { + "partyInvestigationsPerPeriodHint": { + "description": "Required. A number that gives the tuner a hint on the number of parties from this data that will be investigated per period (monthly). This is used to control how the model is evaluated. For example, when trying AML AI for the first time, we recommend setting this to the number of parties investigated in an average month, based on alerts from your existing automated alerting system.", + "type": "string", + "format": "int64" + } + } + }, + "GoogleCloudFinancialservicesV1EngineConfigHyperparameterSource": { + "id": "GoogleCloudFinancialservicesV1EngineConfigHyperparameterSource", + "description": "Parameters for bootstrapping an Engine Config with the results of another one.", + "type": "object", + "properties": { + "sourceEngineConfig": { + "description": "Required. The resource name of the source EngineConfig whose outputs are used. Format: \"/projects/{project_num}/locations/{location}/instances/{instance}/engineConfigs/{engine_config}\"", + "type": "string" + }, + "sourceEngineVersion": { + "description": "Output only. The resource name of the EngineVersion that was used in the tuning run. Format: \"/projects/{project_num}/locations/{location}/instances/{instance}/engineVersions/{engine_version}\"", + "readOnly": true, + "type": "string" + } + } + }, + "GoogleCloudFinancialservicesV1ExportEngineConfigMetadataRequest": { + "id": "GoogleCloudFinancialservicesV1ExportEngineConfigMetadataRequest", + "description": "Request for exporting EngineConfig metadata.", + "type": "object", + "properties": { + "structuredMetadataDestination": { + "description": "Required. BigQuery output where the metadata will be written.", + "$ref": "GoogleCloudFinancialservicesV1BigQueryDestination" + } + } + }, + "GoogleCloudFinancialservicesV1EngineVersion": { + "id": "GoogleCloudFinancialservicesV1EngineVersion", + "description": "EngineVersion controls which version of the engine is used to tune, train, and run the model.", + "type": "object", + "properties": { + "name": { + "description": "Output only. Identifier. The resource name of the EngineVersion format: \"/projects/{project_num}/locations/{location}/instances/{instance}/engineVersions/{engine_version}\"", + "readOnly": true, + "type": "string" + }, + "state": { + "description": "Output only. The state of the version.", + "readOnly": true, + "type": "string", + "enumDescriptions": [ + "Default state, should never be used.", + "Version is available for training and inference.", + "Models using this version can still be run, but new ones cannot be trained.", + "Version is deprecated, listed for informational purposes only." + ], + "enum": [ + "STATE_UNSPECIFIED", + "ACTIVE", + "LIMITED", + "DECOMMISSIONED" + ] + }, + "expectedLimitationStartTime": { + "description": "Output only. Planned time to stop allowing training/tuning using this version. Existing trained models can still be used for prediction/backtest.", + "readOnly": true, + "type": "string", + "format": "google-datetime" + }, + "expectedDecommissionTime": { + "description": "Output only. Planned time to stop supporting the version, in addition to no training or tuning, models trained on this version can no longer be used for prediction/backtest.", + "readOnly": true, + "type": "string", + "format": "google-datetime" + }, + "lineOfBusiness": { + "description": "Output only. The line of business (Retail/Commercial) this engine version is used for.", + "readOnly": true, + "type": "string", + "enumDescriptions": [ + "An unspecified LineOfBusiness. Do not use.", + "Commercial LineOfBusiness.", + "Retail LineOfBusiness." + ], + "enum": [ + "LINE_OF_BUSINESS_UNSPECIFIED", + "COMMERCIAL", + "RETAIL" + ] + } + } + }, + "GoogleCloudFinancialservicesV1ListEngineVersionsResponse": { + "id": "GoogleCloudFinancialservicesV1ListEngineVersionsResponse", + "description": "The response to a list call containng the list of engine versions.", + "type": "object", + "properties": { + "engineVersions": { + "description": "List of EngineVersion resources", + "type": "array", + "items": { + "$ref": "GoogleCloudFinancialservicesV1EngineVersion" + } + }, + "nextPageToken": { + "description": "This token should be passed to the next EngineVersionsRequest to retrieve the next page of EngineVersions (empty indicicates we aer done).", + "type": "string" + }, + "unreachable": { + "description": "Locations that could not be reached.", + "type": "array", + "items": { + "type": "string" + } + } + } + }, + "GoogleCloudFinancialservicesV1ListPredictionResultsResponse": { + "id": "GoogleCloudFinancialservicesV1ListPredictionResultsResponse", + "description": "Response for retrieving a list of PredictionResults", + "type": "object", + "properties": { + "predictionResults": { + "description": "List of PredictionResult resources", + "type": "array", + "items": { + "$ref": "GoogleCloudFinancialservicesV1PredictionResult" + } + }, + "nextPageToken": { + "description": "This token should be passed to the next ListPredictionResultsRequest to retrieve the next page of PredictionResults (empty indicicates we are done).", + "type": "string" + }, + "unreachable": { + "description": "Locations that could not be reached.", + "type": "array", + "items": { + "type": "string" + } + } + } + }, + "GoogleCloudFinancialservicesV1PredictionResult": { + "id": "GoogleCloudFinancialservicesV1PredictionResult", + "description": "PredictionResult is the result of using a model to create predictions.", + "type": "object", + "properties": { + "name": { + "description": "Output only. The resource name of the PredictionResult. format: \"/projects/{project_num}/locations/{location}/instances/{instance}/predictionResults/{prediction_result}\"", + "readOnly": true, + "type": "string" + }, + "createTime": { + "description": "Output only. The timestamp of creation of this resource.", + "readOnly": true, + "type": "string", + "format": "google-datetime" + }, + "updateTime": { + "description": "Output only. The timestamp of the most recent update of this resource.", + "readOnly": true, + "type": "string", + "format": "google-datetime" + }, + "labels": { + "description": "Labels", + "type": "object", + "additionalProperties": { + "type": "string" + } + }, + "state": { + "description": "Output only. State of the PredictionResult (creating, active, deleting, etc.)", + "readOnly": true, + "type": "string", + "enumDescriptions": [ + "State is unspecified, should not occur.", + "The resource has not finished being created.", + "The resource is active/ready to be used.", + "The resource is in the process of being updated.", + "The resource is in the process of being deleted." + ], + "enum": [ + "STATE_UNSPECIFIED", + "CREATING", + "ACTIVE", + "UPDATING", + "DELETING" + ] + }, + "dataset": { + "description": "Required. The resource name of the Dataset to do predictions on Format: \"/projects/{project_num}/locations/{location}/instances/{instance}/dataset/{dataset_id}\"", + "type": "string" + }, + "model": { + "description": "Required. The resource name of the Model to use to use to make predictions Format: \"/projects/{project_num}/locations/{location}/instances/{instance}/models/{model}\"", + "type": "string" + }, + "endTime": { + "description": "Required. Specifies the latest time from which data is used to generate features for predictions. This time should be no later than the end of the date_range of the dataset.", + "type": "string", + "format": "google-datetime" + }, + "predictionPeriods": { + "description": "The number of consecutive months to produce predictions for, ending with the last full month prior to end_time according to the dataset's timezone.", + "type": "integer", + "format": "int32" + }, + "outputs": { + "description": "Required. Where to write the output of the predictions.", + "$ref": "GoogleCloudFinancialservicesV1PredictionResultOutputs" + }, + "lineOfBusiness": { + "description": "Output only. The line of business (Retail/Commercial) this prediction is for. Determined by Model, cannot be set by user.", + "readOnly": true, + "type": "string", + "enumDescriptions": [ + "An unspecified LineOfBusiness. Do not use.", + "Commercial LineOfBusiness.", + "Retail LineOfBusiness." + ], + "enum": [ + "LINE_OF_BUSINESS_UNSPECIFIED", + "COMMERCIAL", + "RETAIL" + ] + } + } + }, + "GoogleCloudFinancialservicesV1PredictionResultOutputs": { + "id": "GoogleCloudFinancialservicesV1PredictionResultOutputs", + "description": "Outputs is a list of places where the operation will write results.", + "type": "object", + "properties": { + "predictionDestination": { + "description": "Required. The location to output the predictions.", + "$ref": "GoogleCloudFinancialservicesV1BigQueryDestination" + }, + "explainabilityDestination": { + "description": "The location to output explainability information. If not specified no explainability data will be output.", + "$ref": "GoogleCloudFinancialservicesV1BigQueryDestination" + } + } + }, + "GoogleCloudFinancialservicesV1ExportPredictionResultMetadataRequest": { + "id": "GoogleCloudFinancialservicesV1ExportPredictionResultMetadataRequest", + "description": "Request for exporting PredictionResult metadata.", + "type": "object", + "properties": { + "structuredMetadataDestination": { + "description": "Required. BigQuery output where the metadata will be written.", + "$ref": "GoogleCloudFinancialservicesV1BigQueryDestination" + } + } + }, + "GoogleCloudFinancialservicesV1ListBacktestResultsResponse": { + "id": "GoogleCloudFinancialservicesV1ListBacktestResultsResponse", + "description": "Response for retrieving a list of BacktestResults", + "type": "object", + "properties": { + "backtestResults": { + "description": "List of BacktestResult resources", + "type": "array", + "items": { + "$ref": "GoogleCloudFinancialservicesV1BacktestResult" + } + }, + "nextPageToken": { + "description": "This token should be passed to the next ListBacktestResultsRequest to retrieve the next page of BacktestResults (empty indicicates we are done).", + "type": "string" + }, + "unreachable": { + "description": "Locations that could not be reached.", + "type": "array", + "items": { + "type": "string" + } + } + } + }, + "GoogleCloudFinancialservicesV1BacktestResult": { + "id": "GoogleCloudFinancialservicesV1BacktestResult", + "description": "BacktestResult is created to test the performance of a model on a dataset.", + "type": "object", + "properties": { + "name": { + "description": "Output only. The resource name of the BacktestResult. format: \"/projects/{project_num}/locations/{location}/instances/{instance}/backtestResults/{backtest_result}\"", + "readOnly": true, + "type": "string" + }, + "createTime": { + "description": "Output only. The timestamp of creation of this resource.", + "readOnly": true, + "type": "string", + "format": "google-datetime" + }, + "updateTime": { + "description": "Output only. The timestamp of the most recent update of this resource.", + "readOnly": true, + "type": "string", + "format": "google-datetime" + }, + "labels": { + "description": "Labels", + "type": "object", + "additionalProperties": { + "type": "string" + } + }, + "state": { + "description": "Output only. State of the BacktestResult (creating, active, deleting, etc.)", + "readOnly": true, + "type": "string", + "enumDescriptions": [ + "State is unspecified, should not occur.", + "The resource has not finished being created.", + "The resource is active/ready to be used.", + "The resource is in the process of being updated.", + "The resource is in the process of being deleted." + ], + "enum": [ + "STATE_UNSPECIFIED", + "CREATING", + "ACTIVE", + "UPDATING", + "DELETING" + ] + }, + "dataset": { + "description": "Required. The resource name of the Dataset to backtest on Format: \"/projects/{project_num}/locations/{location}/instances/{instance}/datasets/{dataset}\"", + "type": "string" + }, + "model": { + "description": "Required. The resource name of the Model to use to use to backtest. Format: \"/projects/{project_num}/locations/{location}/instances/{instance}/models/{model}\"", + "type": "string" + }, + "endTime": { + "description": "Required. End_time specifies the latest time from which labels are used and from which data is used to generate features for backtesting. End_time should be no later than the end of the date_range of the primary dataset.", + "type": "string", + "format": "google-datetime" + }, + "backtestPeriods": { + "description": "The number of consecutive months to conduct backtesting for, ending with the last full month prior to the end_time according to the dataset's timezone.", + "type": "integer", + "format": "int32" + }, + "performanceTarget": { + "description": "Required. PerformanceTarget gives information on how the test will be evaluated.", + "$ref": "GoogleCloudFinancialservicesV1BacktestResultPerformanceTarget" + }, + "lineOfBusiness": { + "description": "Output only. The line of business (Retail/Commercial) this backtest is for. Determined by Model, cannot be set by user.", + "readOnly": true, + "type": "string", + "enumDescriptions": [ + "An unspecified LineOfBusiness. Do not use.", + "Commercial LineOfBusiness.", + "Retail LineOfBusiness." + ], + "enum": [ + "LINE_OF_BUSINESS_UNSPECIFIED", + "COMMERCIAL", + "RETAIL" + ] + } + } + }, + "GoogleCloudFinancialservicesV1BacktestResultPerformanceTarget": { + "id": "GoogleCloudFinancialservicesV1BacktestResultPerformanceTarget", + "description": "PerformanceTarget gives hints on how to evaluate the performance of a model.", + "type": "object", + "properties": { + "partyInvestigationsPerPeriodHint": { + "description": "Required. A number that gives the tuner a hint on the number of parties from this data that will be investigated per period (monthly). This is used to control how the model is evaluated. For example, when trying AML AI for the first time, we recommend setting this to the number of parties investigated in an average month, based on alerts from your existing automated alerting system.", + "type": "string", + "format": "int64" + } + } + }, + "GoogleCloudFinancialservicesV1ExportBacktestResultMetadataRequest": { + "id": "GoogleCloudFinancialservicesV1ExportBacktestResultMetadataRequest", + "description": "Request for exporting BacktestResult metadata.", + "type": "object", + "properties": { + "structuredMetadataDestination": { + "description": "Required. BigQuery output where the metadata will be written.", + "$ref": "GoogleCloudFinancialservicesV1BigQueryDestination" + } + } + }, + "ListLocationsResponse": { + "id": "ListLocationsResponse", + "description": "The response message for Locations.ListLocations.", + "type": "object", + "properties": { + "locations": { + "description": "A list of locations that matches the specified filter in the request.", + "type": "array", + "items": { + "$ref": "Location" + } + }, + "nextPageToken": { + "description": "The standard List next-page token.", + "type": "string" + } + } + }, + "Location": { + "id": "Location", + "description": "A resource that represents a Google Cloud location.", + "type": "object", + "properties": { + "name": { + "description": "Resource name for the location, which may vary between implementations. For example: `\"projects/example-project/locations/us-east1\"`", + "type": "string" + }, + "locationId": { + "description": "The canonical id for this location. For example: `\"us-east1\"`.", + "type": "string" + }, + "displayName": { + "description": "The friendly name for this location, typically a nearby city name. For example, \"Tokyo\".", + "type": "string" + }, + "labels": { + "description": "Cross-service attributes for the location. For example {\"cloud.googleapis.com/region\": \"us-east1\"}", + "type": "object", + "additionalProperties": { + "type": "string" + } + }, + "metadata": { + "description": "Service-specific metadata. For example the available capacity at the given location.", + "type": "object", + "additionalProperties": { + "type": "any", + "description": "Properties of the object. Contains field @type with type URL." + } + } + } + }, + "GoogleCloudFinancialservicesV1OperationMetadata": { + "id": "GoogleCloudFinancialservicesV1OperationMetadata", + "description": "Represents the metadata of the long-running operation.", + "type": "object", + "properties": { + "createTime": { + "description": "Output only. The time the operation was created.", + "readOnly": true, + "type": "string", + "format": "google-datetime" + }, + "endTime": { + "description": "Output only. The time the operation finished running.", + "readOnly": true, + "type": "string", + "format": "google-datetime" + }, + "target": { + "description": "Output only. Server-defined resource path for the target of the operation. format: \"/projects/{project}/...\" VPC-SC restrictions apply.", + "readOnly": true, + "type": "string" + }, + "verb": { + "description": "Output only. Name of the verb executed by the operation.", + "readOnly": true, + "type": "string" + }, + "statusMessage": { + "description": "Output only. Human-readable status of the operation, if any.", + "readOnly": true, + "type": "string" + }, + "requestedCancellation": { + "description": "Output only. Identifies whether the user has requested cancellation of the operation. Successfully cancelled operations have a Operation.error value with a google.rpc.Status.code of 1, corresponding to `Code.CANCELLED`.", + "readOnly": true, + "type": "boolean" + }, + "apiVersion": { + "description": "Output only. API version used to start the operation.", + "readOnly": true, + "type": "string" + } + } + }, + "GoogleCloudFinancialservicesV1ImportRegisteredPartiesResponse": { + "id": "GoogleCloudFinancialservicesV1ImportRegisteredPartiesResponse", + "description": "Response for adding/removing registered parties from BigQuery tables.", + "type": "object", + "properties": { + "partiesAdded": { + "description": "LINT.IfChange Number of parties added by this operation.", + "type": "string", + "format": "int64" + }, + "partiesRemoved": { + "description": "Number of parties removed by this operation.", + "type": "string", + "format": "int64" + }, + "partiesTotal": { + "description": "Total number of parties that are registered in this instance, after the update operation was completed.", + "type": "string", + "format": "int64" + }, + "partiesFailedToRemove": { + "description": "Number of parties that failed to be removed by this operation.", + "type": "string", + "format": "int64" + }, + "partiesUptiered": { + "type": "string", + "format": "int64" + }, + "partiesDowntiered": { + "description": "Total number of parties that are downtiered in this instance", + "type": "string", + "format": "int64" + }, + "partiesFailedToDowntier": { + "description": "Number of parties that failed to be downtiered", + "type": "string", + "format": "int64" + } + } + } + }, + "name": "financialservices", + "servicePath": "", + "basePath": "", + "ownerName": "Google", + "baseUrl": "https://financialservices.googleapis.com/", + "version": "v1", + "id": "financialservices:v1", + "documentationLink": "https://cloud.google.com/solutions/financial-services", + "auth": { + "oauth2": { + "scopes": { + "https://www.googleapis.com/auth/cloud-platform": { + "description": "See, edit, configure, and delete your Google Cloud data and see the email address for your Google Account." + } + } + } + }, + "kind": "discovery#restDescription", + "resources": { + "projects": { + "resources": { + "locations": { + "methods": { + "list": { + "id": "financialservices.projects.locations.list", + "path": "v1/{+name}/locations", + "flatPath": "v1/projects/{projectsId}/locations", + "httpMethod": "GET", + "parameters": { + "name": { + "description": "The resource that owns the locations collection, if applicable.", + "pattern": "^projects/[^/]+$", + "location": "path", + "required": true, + "type": "string" + }, + "filter": { + "description": "A filter to narrow down results to a preferred subset. The filtering language accepts strings like `\"displayName=tokyo\"`, and is documented in more detail in [AIP-160](https://google.aip.dev/160).", + "location": "query", + "type": "string" + }, + "pageSize": { + "description": "The maximum number of results to return. If not set, the service selects a default.", + "location": "query", + "type": "integer", + "format": "int32" + }, + "pageToken": { + "description": "A page token received from the `next_page_token` field in the response. Send that page token to receive the subsequent page.", + "location": "query", + "type": "string" + } + }, + "parameterOrder": [ + "name" + ], + "response": { + "$ref": "ListLocationsResponse" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "Lists information about the supported locations for this service." + }, + "get": { + "id": "financialservices.projects.locations.get", + "path": "v1/{+name}", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}", + "httpMethod": "GET", + "parameters": { + "name": { + "description": "Resource name for the location.", + "pattern": "^projects/[^/]+/locations/[^/]+$", + "location": "path", + "required": true, + "type": "string" + } + }, + "parameterOrder": [ + "name" + ], + "response": { + "$ref": "Location" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "Gets information about a location." + } + }, + "resources": { + "operations": { + "methods": { + "list": { + "id": "financialservices.projects.locations.operations.list", + "path": "v1/{+name}/operations", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}/operations", + "httpMethod": "GET", + "parameters": { + "name": { + "description": "The name of the operation's parent resource.", + "pattern": "^projects/[^/]+/locations/[^/]+$", + "location": "path", + "required": true, + "type": "string" + }, + "filter": { + "description": "The standard list filter.", + "location": "query", + "type": "string" + }, + "pageSize": { + "description": "The standard list page size.", + "location": "query", + "type": "integer", + "format": "int32" + }, + "pageToken": { + "description": "The standard list page token.", + "location": "query", + "type": "string" + } + }, + "parameterOrder": [ + "name" + ], + "response": { + "$ref": "ListOperationsResponse" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "Lists operations that match the specified filter in the request. If the server doesn't support this method, it returns `UNIMPLEMENTED`." + }, + "get": { + "id": "financialservices.projects.locations.operations.get", + "path": "v1/{+name}", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}/operations/{operationsId}", + "httpMethod": "GET", + "parameters": { + "name": { + "description": "The name of the operation resource.", + "pattern": "^projects/[^/]+/locations/[^/]+/operations/[^/]+$", + "location": "path", + "required": true, + "type": "string" + } + }, + "parameterOrder": [ + "name" + ], + "response": { + "$ref": "Operation" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "Gets the latest state of a long-running operation. Clients can use this method to poll the operation result at intervals as recommended by the API service." + }, + "delete": { + "id": "financialservices.projects.locations.operations.delete", + "path": "v1/{+name}", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}/operations/{operationsId}", + "httpMethod": "DELETE", + "parameters": { + "name": { + "description": "The name of the operation resource to be deleted.", + "pattern": "^projects/[^/]+/locations/[^/]+/operations/[^/]+$", + "location": "path", + "required": true, + "type": "string" + } + }, + "parameterOrder": [ + "name" + ], + "response": { + "$ref": "Empty" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "Deletes a long-running operation. This method indicates that the client is no longer interested in the operation result. It does not cancel the operation. If the server doesn't support this method, it returns `google.rpc.Code.UNIMPLEMENTED`." + }, + "cancel": { + "id": "financialservices.projects.locations.operations.cancel", + "path": "v1/{+name}:cancel", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}/operations/{operationsId}:cancel", + "httpMethod": "POST", + "parameters": { + "name": { + "description": "The name of the operation resource to be cancelled.", + "pattern": "^projects/[^/]+/locations/[^/]+/operations/[^/]+$", + "location": "path", + "required": true, + "type": "string" + } + }, + "parameterOrder": [ + "name" + ], + "request": { + "$ref": "CancelOperationRequest" + }, + "response": { + "$ref": "Empty" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "Starts asynchronous cancellation on a long-running operation. The server makes a best effort to cancel the operation, but success is not guaranteed. If the server doesn't support this method, it returns `google.rpc.Code.UNIMPLEMENTED`. Clients can use Operations.GetOperation or other methods to check whether the cancellation succeeded or whether the operation completed despite cancellation. On successful cancellation, the operation is not deleted; instead, it becomes an operation with an Operation.error value with a google.rpc.Status.code of 1, corresponding to `Code.CANCELLED`." + } + } + }, + "instances": { + "methods": { + "list": { + "id": "financialservices.projects.locations.instances.list", + "path": "v1/{+parent}/instances", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}/instances", + "httpMethod": "GET", + "parameters": { + "parent": { + "description": "Required. The parent of the Instance is the location for that Instance. Every location has exactly one instance.", + "pattern": "^projects/[^/]+/locations/[^/]+$", + "location": "path", + "required": true, + "type": "string" + }, + "pageSize": { + "description": "The number of resources to be included in the response. The response contains a next_page_token, which can be used to retrieve the next page of resources.", + "location": "query", + "type": "integer", + "format": "int32" + }, + "pageToken": { + "description": "In case of paginated results, this is the token that was returned in the previous ListInstancesResponse. It should be copied here to retrieve the next page of resources. This will be empty for the first instance of ListInstancesRequest.", + "location": "query", + "type": "string" + }, + "filter": { + "description": "Specify a filter to narrow search results.", + "location": "query", + "type": "string" + }, + "orderBy": { + "description": "Specify a field to use for ordering.", + "location": "query", + "type": "string" + } + }, + "parameterOrder": [ + "parent" + ], + "response": { + "$ref": "GoogleCloudFinancialservicesV1ListInstancesResponse" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "Lists instances." + }, + "get": { + "id": "financialservices.projects.locations.instances.get", + "path": "v1/{+name}", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}/instances/{instancesId}", + "httpMethod": "GET", + "parameters": { + "name": { + "description": "Required. The resource name of the Instance.", + "pattern": "^projects/[^/]+/locations/[^/]+/instances/[^/]+$", + "location": "path", + "required": true, + "type": "string" + } + }, + "parameterOrder": [ + "name" + ], + "response": { + "$ref": "GoogleCloudFinancialservicesV1Instance" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "Gets an instance." + }, + "create": { + "id": "financialservices.projects.locations.instances.create", + "path": "v1/{+parent}/instances", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}/instances", + "httpMethod": "POST", + "parameters": { + "parent": { + "description": "Required. The parent of the Instance is the location for that Instance. Every location has exactly one instance.", + "pattern": "^projects/[^/]+/locations/[^/]+$", + "location": "path", + "required": true, + "type": "string" + }, + "instanceId": { + "description": "Required. The resource id of the instance.", + "location": "query", + "type": "string" + }, + "requestId": { + "description": "Optional. An optional request ID to identify requests. Specify a unique request ID so that if you must retry your request, the server will know to ignore the request if it has already been completed. The server will guarantee that for at least 60 minutes since the first request. For example, consider a situation where you make an initial request and the request times out. If you make the request again with the same request ID, the server can check if original operation with the same request ID was received, and if so, will ignore the second request. This prevents clients from accidentally creating duplicate commitments. The request ID must be a valid UUID with the exception that zero UUID is not supported (00000000-0000-0000-0000-000000000000).", + "location": "query", + "type": "string" + } + }, + "parameterOrder": [ + "parent" + ], + "request": { + "$ref": "GoogleCloudFinancialservicesV1Instance" + }, + "response": { + "$ref": "Operation" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "Creates an instance." + }, + "patch": { + "id": "financialservices.projects.locations.instances.patch", + "path": "v1/{+name}", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}/instances/{instancesId}", + "httpMethod": "PATCH", + "parameters": { + "name": { + "description": "Output only. The full path to the Instance resource in this API. format: \"projects/{project}/locations/{location}/instances/{instance}\"", + "pattern": "^projects/[^/]+/locations/[^/]+/instances/[^/]+$", + "location": "path", + "required": true, + "type": "string" + }, + "updateMask": { + "description": "Optional. Field mask is used to specify the fields to be overwritten in the Instance resource by the update. The fields specified in the update_mask are relative to the resource, not the full request. A field will be overwritten if it is in the mask. If the user does not provide a mask then all fields will be overwritten.", + "location": "query", + "type": "string", + "format": "google-fieldmask" + }, + "requestId": { + "description": "Optional. An optional request ID to identify requests. Specify a unique request ID so that if you must retry your request, the server will know to ignore the request if it has already been completed. The server will guarantee that for at least 60 minutes since the first request. For example, consider a situation where you make an initial request and the request times out. If you make the request again with the same request ID, the server can check if original operation with the same request ID was received, and if so, will ignore the second request. This prevents clients from accidentally creating duplicate commitments. The request ID must be a valid UUID with the exception that zero UUID is not supported (00000000-0000-0000-0000-000000000000).", + "location": "query", + "type": "string" + } + }, + "parameterOrder": [ + "name" + ], + "request": { + "$ref": "GoogleCloudFinancialservicesV1Instance" + }, + "response": { + "$ref": "Operation" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "Updates the parameters of a single Instance." + }, + "delete": { + "id": "financialservices.projects.locations.instances.delete", + "path": "v1/{+name}", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}/instances/{instancesId}", + "httpMethod": "DELETE", + "parameters": { + "name": { + "description": "Required. The resource name of the Instance.", + "pattern": "^projects/[^/]+/locations/[^/]+/instances/[^/]+$", + "location": "path", + "required": true, + "type": "string" + }, + "requestId": { + "description": "Optional. An optional request ID to identify requests. Specify a unique request ID so that if you must retry your request, the server will know to ignore the request if it has already been completed. The server will guarantee that for at least 60 minutes after the first request. For example, consider a situation where you make an initial request and the request times out. If you make the request again with the same request ID, the server can check if original operation with the same request ID was received, and if so, will ignore the second request. This prevents clients from accidentally creating duplicate commitments. The request ID must be a valid UUID with the exception that zero UUID is not supported (00000000-0000-0000-0000-000000000000).", + "location": "query", + "type": "string" + } + }, + "parameterOrder": [ + "name" + ], + "response": { + "$ref": "Operation" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "Deletes an instance." + }, + "importRegisteredParties": { + "id": "financialservices.projects.locations.instances.importRegisteredParties", + "path": "v1/{+name}:importRegisteredParties", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}/instances/{instancesId}:importRegisteredParties", + "httpMethod": "POST", + "parameters": { + "name": { + "description": "Required. The full path to the Instance resource in this API. format: \"projects/{project}/locations/{location}/instances/{instance}\"", + "pattern": "^projects/[^/]+/locations/[^/]+/instances/[^/]+$", + "location": "path", + "required": true, + "type": "string" + } + }, + "parameterOrder": [ + "name" + ], + "request": { + "$ref": "GoogleCloudFinancialservicesV1ImportRegisteredPartiesRequest" + }, + "response": { + "$ref": "Operation" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "Imports the list of registered parties. See [Create and manage instances](https://cloud.google.com/financial-services/anti-money-laundering/docs/create-and-manage-instances#import-registered-parties) for information on the input schema and response for this method." + }, + "exportRegisteredParties": { + "id": "financialservices.projects.locations.instances.exportRegisteredParties", + "path": "v1/{+name}:exportRegisteredParties", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}/instances/{instancesId}:exportRegisteredParties", + "httpMethod": "POST", + "parameters": { + "name": { + "description": "Required. The full path to the Instance resource in this API. format: \"projects/{project}/locations/{location}/instances/{instance}\"", + "pattern": "^projects/[^/]+/locations/[^/]+/instances/[^/]+$", + "location": "path", + "required": true, + "type": "string" + } + }, + "parameterOrder": [ + "name" + ], + "request": { + "$ref": "GoogleCloudFinancialservicesV1ExportRegisteredPartiesRequest" + }, + "response": { + "$ref": "Operation" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "Exports the list of registered parties. See [Create and manage instances](https://cloud.google.com/financial-services/anti-money-laundering/docs/create-and-manage-instances#export-registered-parties) for information on the output schema for this method." + } + }, + "resources": { + "datasets": { + "methods": { + "list": { + "id": "financialservices.projects.locations.instances.datasets.list", + "path": "v1/{+parent}/datasets", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}/instances/{instancesId}/datasets", + "httpMethod": "GET", + "parameters": { + "parent": { + "description": "Required. The parent of the Dataset is the Instance.", + "pattern": "^projects/[^/]+/locations/[^/]+/instances/[^/]+$", + "location": "path", + "required": true, + "type": "string" + }, + "pageSize": { + "description": "The number of resources to be included in the response. The response contains a next_page_token, which can be used to retrieve the next page of resources.", + "location": "query", + "type": "integer", + "format": "int32" + }, + "pageToken": { + "description": "In case of paginated results, this is the token that was returned in the previous ListDatasetResponse. It should be copied here to retrieve the next page of resources. Empty will give the first page of ListDatasetRequest, and the last page will return an empty page_token.", + "location": "query", + "type": "string" + }, + "filter": { + "description": "Specify a filter to narrow search results.", + "location": "query", + "type": "string" + }, + "orderBy": { + "description": "Specify a field to use for ordering.", + "location": "query", + "type": "string" + } + }, + "parameterOrder": [ + "parent" + ], + "response": { + "$ref": "GoogleCloudFinancialservicesV1ListDatasetsResponse" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "Lists datasets." + }, + "get": { + "id": "financialservices.projects.locations.instances.datasets.get", + "path": "v1/{+name}", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}/instances/{instancesId}/datasets/{datasetsId}", + "httpMethod": "GET", + "parameters": { + "name": { + "description": "Required. The resource name of the Dataset", + "pattern": "^projects/[^/]+/locations/[^/]+/instances/[^/]+/datasets/[^/]+$", + "location": "path", + "required": true, + "type": "string" + } + }, + "parameterOrder": [ + "name" + ], + "response": { + "$ref": "GoogleCloudFinancialservicesV1Dataset" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "Gets a dataset." + }, + "create": { + "id": "financialservices.projects.locations.instances.datasets.create", + "path": "v1/{+parent}/datasets", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}/instances/{instancesId}/datasets", + "httpMethod": "POST", + "parameters": { + "parent": { + "description": "Required. The parent of the Dataset is the Instance.", + "pattern": "^projects/[^/]+/locations/[^/]+/instances/[^/]+$", + "location": "path", + "required": true, + "type": "string" + }, + "datasetId": { + "description": "Required. The resource id of the dataset", + "location": "query", + "type": "string" + }, + "requestId": { + "description": "Optional. An optional request ID to identify requests. Specify a unique request ID so that if you must retry your request, the server will know to ignore the request if it has already been completed. The server will guarantee that for at least 60 minutes since the first request. For example, consider a situation where you make an initial request and the request times out. If you make the request again with the same request ID, the server can check if original operation with the same request ID was received, and if so, will ignore the second request. This prevents clients from accidentally creating duplicate commitments. The request ID must be a valid UUID with the exception that zero UUID is not supported (00000000-0000-0000-0000-000000000000).", + "location": "query", + "type": "string" + } + }, + "parameterOrder": [ + "parent" + ], + "request": { + "$ref": "GoogleCloudFinancialservicesV1Dataset" + }, + "response": { + "$ref": "Operation" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "Creates a dataset." + }, + "patch": { + "id": "financialservices.projects.locations.instances.datasets.patch", + "path": "v1/{+name}", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}/instances/{instancesId}/datasets/{datasetsId}", + "httpMethod": "PATCH", + "parameters": { + "name": { + "description": "Output only. The resource name of the Dataset. format: \"/projects/{project_num}/locations/{location}/instances/{instance}/datasets/{dataset}\"", + "pattern": "^projects/[^/]+/locations/[^/]+/instances/[^/]+/datasets/[^/]+$", + "location": "path", + "required": true, + "type": "string" + }, + "updateMask": { + "description": "Optional. Field mask is used to specify the fields to be overwritten in the Dataset resource by the update. The fields specified in the update_mask are relative to the resource, not the full request. A field will be overwritten if it is in the mask. If the user does not provide a mask then all fields will be overwritten.", + "location": "query", + "type": "string", + "format": "google-fieldmask" + }, + "requestId": { + "description": "Optional. An optional request ID to identify requests. Specify a unique request ID so that if you must retry your request, the server will know to ignore the request if it has already been completed. The server will guarantee that for at least 60 minutes since the first request. For example, consider a situation where you make an initial request and the request times out. If you make the request again with the same request ID, the server can check if original operation with the same request ID was received, and if so, will ignore the second request. This prevents clients from accidentally creating duplicate commitments. The request ID must be a valid UUID with the exception that zero UUID is not supported (00000000-0000-0000-0000-000000000000).", + "location": "query", + "type": "string" + } + }, + "parameterOrder": [ + "name" + ], + "request": { + "$ref": "GoogleCloudFinancialservicesV1Dataset" + }, + "response": { + "$ref": "Operation" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "Updates the parameters of a single Dataset." + }, + "delete": { + "id": "financialservices.projects.locations.instances.datasets.delete", + "path": "v1/{+name}", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}/instances/{instancesId}/datasets/{datasetsId}", + "httpMethod": "DELETE", + "parameters": { + "name": { + "description": "Required. The resource name of the Dataset.", + "pattern": "^projects/[^/]+/locations/[^/]+/instances/[^/]+/datasets/[^/]+$", + "location": "path", + "required": true, + "type": "string" + }, + "requestId": { + "description": "Optional. An optional request ID to identify requests. Specify a unique request ID so that if you must retry your request, the server will know to ignore the request if it has already been completed. The server will guarantee that for at least 60 minutes after the first request. For example, consider a situation where you make an initial request and the request times out. If you make the request again with the same request ID, the server can check if original operation with the same request ID was received, and if so, will ignore the second request. This prevents clients from accidentally creating duplicate commitments. The request ID must be a valid UUID with the exception that zero UUID is not supported (00000000-0000-0000-0000-000000000000).", + "location": "query", + "type": "string" + } + }, + "parameterOrder": [ + "name" + ], + "response": { + "$ref": "Operation" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "Deletes a dataset." + } + } + }, + "models": { + "methods": { + "list": { + "id": "financialservices.projects.locations.instances.models.list", + "path": "v1/{+parent}/models", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}/instances/{instancesId}/models", + "httpMethod": "GET", + "parameters": { + "parent": { + "description": "Required. The parent of the Model is the Instance.", + "pattern": "^projects/[^/]+/locations/[^/]+/instances/[^/]+$", + "location": "path", + "required": true, + "type": "string" + }, + "pageSize": { + "description": "The number of resources to be included in the response. The response contains a next_page_token, which can be used to retrieve the next page of resources.", + "location": "query", + "type": "integer", + "format": "int32" + }, + "pageToken": { + "description": "In case of paginated results, this is the token that was returned in the previous ListModelsResponse. It should be copied here to retrieve the next page of resources. Empty will give the first page of ListModelsRequest, and the last page will return an empty page_token.", + "location": "query", + "type": "string" + }, + "filter": { + "description": "Specify a filter to narrow search results.", + "location": "query", + "type": "string" + }, + "orderBy": { + "description": "Specify a field to use for ordering.", + "location": "query", + "type": "string" + } + }, + "parameterOrder": [ + "parent" + ], + "response": { + "$ref": "GoogleCloudFinancialservicesV1ListModelsResponse" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "Lists models." + }, + "get": { + "id": "financialservices.projects.locations.instances.models.get", + "path": "v1/{+name}", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}/instances/{instancesId}/models/{modelsId}", + "httpMethod": "GET", + "parameters": { + "name": { + "description": "Required. The resource name of the Model", + "pattern": "^projects/[^/]+/locations/[^/]+/instances/[^/]+/models/[^/]+$", + "location": "path", + "required": true, + "type": "string" + } + }, + "parameterOrder": [ + "name" + ], + "response": { + "$ref": "GoogleCloudFinancialservicesV1Model" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "Gets a model." + }, + "create": { + "id": "financialservices.projects.locations.instances.models.create", + "path": "v1/{+parent}/models", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}/instances/{instancesId}/models", + "httpMethod": "POST", + "parameters": { + "parent": { + "description": "Required. The parent of the Model is the Instance.", + "pattern": "^projects/[^/]+/locations/[^/]+/instances/[^/]+$", + "location": "path", + "required": true, + "type": "string" + }, + "modelId": { + "description": "Required. The resource id of the Model", + "location": "query", + "type": "string" + }, + "requestId": { + "description": "Optional. An optional request ID to identify requests. Specify a unique request ID so that if you must retry your request, the server will know to ignore the request if it has already been completed. The server will guarantee that for at least 60 minutes since the first request. For example, consider a situation where you make an initial request and the request times out. If you make the request again with the same request ID, the server can check if original operation with the same request ID was received, and if so, will ignore the second request. This prevents clients from accidentally creating duplicate commitments. The request ID must be a valid UUID with the exception that zero UUID is not supported (00000000-0000-0000-0000-000000000000).", + "location": "query", + "type": "string" + } + }, + "parameterOrder": [ + "parent" + ], + "request": { + "$ref": "GoogleCloudFinancialservicesV1Model" + }, + "response": { + "$ref": "Operation" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "Creates a model." + }, + "patch": { + "id": "financialservices.projects.locations.instances.models.patch", + "path": "v1/{+name}", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}/instances/{instancesId}/models/{modelsId}", + "httpMethod": "PATCH", + "parameters": { + "name": { + "description": "Output only. The resource name of the Model. format: \"/projects/{project_num}/locations/{location}/instances/{instance}/models/{model}\"", + "pattern": "^projects/[^/]+/locations/[^/]+/instances/[^/]+/models/[^/]+$", + "location": "path", + "required": true, + "type": "string" + }, + "updateMask": { + "description": "Optional. Field mask is used to specify the fields to be overwritten in the Model resource by the update. The fields specified in the update_mask are relative to the resource, not the full request. A field will be overwritten if it is in the mask. If the user does not provide a mask then all fields will be overwritten.", + "location": "query", + "type": "string", + "format": "google-fieldmask" + }, + "requestId": { + "description": "Optional. An optional request ID to identify requests. Specify a unique request ID so that if you must retry your request, the server will know to ignore the request if it has already been completed. The server will guarantee that for at least 60 minutes since the first request. For example, consider a situation where you make an initial request and the request times out. If you make the request again with the same request ID, the server can check if original operation with the same request ID was received, and if so, will ignore the second request. This prevents clients from accidentally creating duplicate commitments. The request ID must be a valid UUID with the exception that zero UUID is not supported (00000000-0000-0000-0000-000000000000).", + "location": "query", + "type": "string" + } + }, + "parameterOrder": [ + "name" + ], + "request": { + "$ref": "GoogleCloudFinancialservicesV1Model" + }, + "response": { + "$ref": "Operation" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "Updates the parameters of a single Model." + }, + "exportMetadata": { + "id": "financialservices.projects.locations.instances.models.exportMetadata", + "path": "v1/{+model}:exportMetadata", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}/instances/{instancesId}/models/{modelsId}:exportMetadata", + "httpMethod": "POST", + "parameters": { + "model": { + "description": "Required. The resource name of the Model.", + "pattern": "^projects/[^/]+/locations/[^/]+/instances/[^/]+/models/[^/]+$", + "location": "path", + "required": true, + "type": "string" + } + }, + "parameterOrder": [ + "model" + ], + "request": { + "$ref": "GoogleCloudFinancialservicesV1ExportModelMetadataRequest" + }, + "response": { + "$ref": "Operation" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "Export governance information for a Model resource. For information on the exported fields, see [AML output data model](https://cloud.google.com/financial-services/anti-money-laundering/docs/reference/schemas/aml-output-data-model#model)." + }, + "delete": { + "id": "financialservices.projects.locations.instances.models.delete", + "path": "v1/{+name}", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}/instances/{instancesId}/models/{modelsId}", + "httpMethod": "DELETE", + "parameters": { + "name": { + "description": "Required. The resource name of the Model.", + "pattern": "^projects/[^/]+/locations/[^/]+/instances/[^/]+/models/[^/]+$", + "location": "path", + "required": true, + "type": "string" + }, + "requestId": { + "description": "Optional. An optional request ID to identify requests. Specify a unique request ID so that if you must retry your request, the server will know to ignore the request if it has already been completed. The server will guarantee that for at least 60 minutes after the first request. For example, consider a situation where you make an initial request and the request times out. If you make the request again with the same request ID, the server can check if original operation with the same request ID was received, and if so, will ignore the second request. This prevents clients from accidentally creating duplicate commitments. The request ID must be a valid UUID with the exception that zero UUID is not supported (00000000-0000-0000-0000-000000000000).", + "location": "query", + "type": "string" + } + }, + "parameterOrder": [ + "name" + ], + "response": { + "$ref": "Operation" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "Deletes a model." + } + } + }, + "engineConfigs": { + "methods": { + "list": { + "id": "financialservices.projects.locations.instances.engineConfigs.list", + "path": "v1/{+parent}/engineConfigs", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}/instances/{instancesId}/engineConfigs", + "httpMethod": "GET", + "parameters": { + "parent": { + "description": "Required. The parent of the EngineConfig is the Instance.", + "pattern": "^projects/[^/]+/locations/[^/]+/instances/[^/]+$", + "location": "path", + "required": true, + "type": "string" + }, + "pageSize": { + "description": "The number of resources to be included in the response. The response contains a next_page_token, which can be used to retrieve the next page of resources.", + "location": "query", + "type": "integer", + "format": "int32" + }, + "pageToken": { + "description": "In case of paginated results, this is the token that was returned in the previous ListEngineConfigsResponse. It should be copied here to retrieve the next page of resources. Empty will give the first page of ListEngineConfigsRequest, and the last page will return an empty page_token.", + "location": "query", + "type": "string" + }, + "filter": { + "description": "Specify a filter to narrow search results.", + "location": "query", + "type": "string" + }, + "orderBy": { + "description": "Specify a field to use for ordering.", + "location": "query", + "type": "string" + } + }, + "parameterOrder": [ + "parent" + ], + "response": { + "$ref": "GoogleCloudFinancialservicesV1ListEngineConfigsResponse" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "Lists engine configs." + }, + "get": { + "id": "financialservices.projects.locations.instances.engineConfigs.get", + "path": "v1/{+name}", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}/instances/{instancesId}/engineConfigs/{engineConfigsId}", + "httpMethod": "GET", + "parameters": { + "name": { + "description": "Required. The resource name of the EngineConfig", + "pattern": "^projects/[^/]+/locations/[^/]+/instances/[^/]+/engineConfigs/[^/]+$", + "location": "path", + "required": true, + "type": "string" + } + }, + "parameterOrder": [ + "name" + ], + "response": { + "$ref": "GoogleCloudFinancialservicesV1EngineConfig" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "Gets an engine config." + }, + "create": { + "id": "financialservices.projects.locations.instances.engineConfigs.create", + "path": "v1/{+parent}/engineConfigs", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}/instances/{instancesId}/engineConfigs", + "httpMethod": "POST", + "parameters": { + "parent": { + "description": "Required. The parent of the EngineConfig is the Instance.", + "pattern": "^projects/[^/]+/locations/[^/]+/instances/[^/]+$", + "location": "path", + "required": true, + "type": "string" + }, + "engineConfigId": { + "description": "Required. The resource id of the EngineConfig", + "location": "query", + "type": "string" + }, + "requestId": { + "description": "Optional. An optional request ID to identify requests. Specify a unique request ID so that if you must retry your request, the server will know to ignore the request if it has already been completed. The server will guarantee that for at least 60 minutes since the first request. For example, consider a situation where you make an initial request and the request times out. If you make the request again with the same request ID, the server can check if original operation with the same request ID was received, and if so, will ignore the second request. This prevents clients from accidentally creating duplicate commitments. The request ID must be a valid UUID with the exception that zero UUID is not supported (00000000-0000-0000-0000-000000000000).", + "location": "query", + "type": "string" + } + }, + "parameterOrder": [ + "parent" + ], + "request": { + "$ref": "GoogleCloudFinancialservicesV1EngineConfig" + }, + "response": { + "$ref": "Operation" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "Creates an engine config." + }, + "patch": { + "id": "financialservices.projects.locations.instances.engineConfigs.patch", + "path": "v1/{+name}", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}/instances/{instancesId}/engineConfigs/{engineConfigsId}", + "httpMethod": "PATCH", + "parameters": { + "name": { + "description": "Output only. The resource name of the EngineConfig. format: \"/projects/{project_num}/locations/{location}/instances/{instance}/engineConfigs/{engine_config}\"", + "pattern": "^projects/[^/]+/locations/[^/]+/instances/[^/]+/engineConfigs/[^/]+$", + "location": "path", + "required": true, + "type": "string" + }, + "updateMask": { + "description": "Optional. Field mask is used to specify the fields to be overwritten in the EngineConfig resource by the update. The fields specified in the update_mask are relative to the resource, not the full request. A field will be overwritten if it is in the mask. If the user does not provide a mask then all fields will be overwritten.", + "location": "query", + "type": "string", + "format": "google-fieldmask" + }, + "requestId": { + "description": "Optional. An optional request ID to identify requests. Specify a unique request ID so that if you must retry your request, the server will know to ignore the request if it has already been completed. The server will guarantee that for at least 60 minutes since the first request. For example, consider a situation where you make an initial request and the request times out. If you make the request again with the same request ID, the server can check if original operation with the same request ID was received, and if so, will ignore the second request. This prevents clients from accidentally creating duplicate commitments. The request ID must be a valid UUID with the exception that zero UUID is not supported (00000000-0000-0000-0000-000000000000).", + "location": "query", + "type": "string" + } + }, + "parameterOrder": [ + "name" + ], + "request": { + "$ref": "GoogleCloudFinancialservicesV1EngineConfig" + }, + "response": { + "$ref": "Operation" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "Updates the parameters of a single EngineConfig." + }, + "exportMetadata": { + "id": "financialservices.projects.locations.instances.engineConfigs.exportMetadata", + "path": "v1/{+engineConfig}:exportMetadata", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}/instances/{instancesId}/engineConfigs/{engineConfigsId}:exportMetadata", + "httpMethod": "POST", + "parameters": { + "engineConfig": { + "description": "Required. The resource name of the EngineConfig.", + "pattern": "^projects/[^/]+/locations/[^/]+/instances/[^/]+/engineConfigs/[^/]+$", + "location": "path", + "required": true, + "type": "string" + } + }, + "parameterOrder": [ + "engineConfig" + ], + "request": { + "$ref": "GoogleCloudFinancialservicesV1ExportEngineConfigMetadataRequest" + }, + "response": { + "$ref": "Operation" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "Export governance information for an EngineConfig resource. For information on the exported fields, see [AML output data model](https://cloud.google.com/financial-services/anti-money-laundering/docs/reference/schemas/aml-output-data-model#engine-config)." + }, + "delete": { + "id": "financialservices.projects.locations.instances.engineConfigs.delete", + "path": "v1/{+name}", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}/instances/{instancesId}/engineConfigs/{engineConfigsId}", + "httpMethod": "DELETE", + "parameters": { + "name": { + "description": "Required. The resource name of the EngineConfig.", + "pattern": "^projects/[^/]+/locations/[^/]+/instances/[^/]+/engineConfigs/[^/]+$", + "location": "path", + "required": true, + "type": "string" + }, + "requestId": { + "description": "Optional. An optional request ID to identify requests. Specify a unique request ID so that if you must retry your request, the server will know to ignore the request if it has already been completed. The server will guarantee that for at least 60 minutes after the first request. For example, consider a situation where you make an initial request and the request times out. If you make the request again with the same request ID, the server can check if original operation with the same request ID was received, and if so, will ignore the second request. This prevents clients from accidentally creating duplicate commitments. The request ID must be a valid UUID with the exception that zero UUID is not supported (00000000-0000-0000-0000-000000000000).", + "location": "query", + "type": "string" + } + }, + "parameterOrder": [ + "name" + ], + "response": { + "$ref": "Operation" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "Deletes an engine config." + } + } + }, + "engineVersions": { + "methods": { + "get": { + "id": "financialservices.projects.locations.instances.engineVersions.get", + "path": "v1/{+name}", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}/instances/{instancesId}/engineVersions/{engineVersionsId}", + "httpMethod": "GET", + "parameters": { + "name": { + "description": "Required. The resource name of the EngineVersion", + "pattern": "^projects/[^/]+/locations/[^/]+/instances/[^/]+/engineVersions/[^/]+$", + "location": "path", + "required": true, + "type": "string" + } + }, + "parameterOrder": [ + "name" + ], + "response": { + "$ref": "GoogleCloudFinancialservicesV1EngineVersion" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "Gets a single EngineVersion." + }, + "list": { + "id": "financialservices.projects.locations.instances.engineVersions.list", + "path": "v1/{+parent}/engineVersions", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}/instances/{instancesId}/engineVersions", + "httpMethod": "GET", + "parameters": { + "parent": { + "description": "Required. The parent of the EngineVersion is the Instance.", + "pattern": "^projects/[^/]+/locations/[^/]+/instances/[^/]+$", + "location": "path", + "required": true, + "type": "string" + }, + "pageSize": { + "description": "Optional. The number of resources to be included in the response. The response contains a next_page_token, which can be used to retrieve the next page of resources.", + "location": "query", + "type": "integer", + "format": "int32" + }, + "pageToken": { + "description": "Optional. In case of paginated results, this is the token that was returned in the previous ListEngineVersionsResponse. It should be copied here to retrieve the next page of resources. Empty will give the first page of ListEngineVersionsRequest, and the last page will return an empty page_token.", + "location": "query", + "type": "string" + }, + "filter": { + "description": "Optional. Specify a filter to narrow search results. If empty or unset will default to \"state!=DEPRECATED\", to view deprecated versions use \"state:*\" or any other filter.", + "location": "query", + "type": "string" + }, + "orderBy": { + "description": "Optional. Specify a field to use for ordering.", + "location": "query", + "type": "string" + } + }, + "parameterOrder": [ + "parent" + ], + "response": { + "$ref": "GoogleCloudFinancialservicesV1ListEngineVersionsResponse" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "Lists EngineVersions for given location." + } + } + }, + "predictionResults": { + "methods": { + "list": { + "id": "financialservices.projects.locations.instances.predictionResults.list", + "path": "v1/{+parent}/predictionResults", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}/instances/{instancesId}/predictionResults", + "httpMethod": "GET", + "parameters": { + "parent": { + "description": "Required. The parent of the PredictionResult is the Instance.", + "pattern": "^projects/[^/]+/locations/[^/]+/instances/[^/]+$", + "location": "path", + "required": true, + "type": "string" + }, + "pageSize": { + "description": "The number of resources to be included in the response. The response contains a next_page_token, which can be used to retrieve the next page of resources.", + "location": "query", + "type": "integer", + "format": "int32" + }, + "pageToken": { + "description": "In case of paginated results, this is the token that was returned in the previous ListPredictionResultsResponse. It should be copied here to retrieve the next page of resources. Empty will give the first page of ListPredictionResultsRequest, and the last page will return an empty page_token.", + "location": "query", + "type": "string" + }, + "filter": { + "description": "Specify a filter to narrow search results.", + "location": "query", + "type": "string" + }, + "orderBy": { + "description": "Specify a field to use for ordering.", + "location": "query", + "type": "string" + } + }, + "parameterOrder": [ + "parent" + ], + "response": { + "$ref": "GoogleCloudFinancialservicesV1ListPredictionResultsResponse" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "List PredictionResults." + }, + "get": { + "id": "financialservices.projects.locations.instances.predictionResults.get", + "path": "v1/{+name}", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}/instances/{instancesId}/predictionResults/{predictionResultsId}", + "httpMethod": "GET", + "parameters": { + "name": { + "description": "Required. The resource name of the PredictionResult", + "pattern": "^projects/[^/]+/locations/[^/]+/instances/[^/]+/predictionResults/[^/]+$", + "location": "path", + "required": true, + "type": "string" + } + }, + "parameterOrder": [ + "name" + ], + "response": { + "$ref": "GoogleCloudFinancialservicesV1PredictionResult" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "Gets a PredictionResult." + }, + "create": { + "id": "financialservices.projects.locations.instances.predictionResults.create", + "path": "v1/{+parent}/predictionResults", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}/instances/{instancesId}/predictionResults", + "httpMethod": "POST", + "parameters": { + "parent": { + "description": "Required. The parent of the PredictionResult is the Instance.", + "pattern": "^projects/[^/]+/locations/[^/]+/instances/[^/]+$", + "location": "path", + "required": true, + "type": "string" + }, + "predictionResultId": { + "description": "Required. The resource id of the PredictionResult", + "location": "query", + "type": "string" + }, + "requestId": { + "description": "Optional. An optional request ID to identify requests. Specify a unique request ID so that if you must retry your request, the server will know to ignore the request if it has already been completed. The server will guarantee that for at least 60 minutes since the first request. For example, consider a situation where you make an initial request and the request times out. If you make the request again with the same request ID, the server can check if original operation with the same request ID was received, and if so, will ignore the second request. This prevents clients from accidentally creating duplicate commitments. The request ID must be a valid UUID with the exception that zero UUID is not supported (00000000-0000-0000-0000-000000000000).", + "location": "query", + "type": "string" + } + }, + "parameterOrder": [ + "parent" + ], + "request": { + "$ref": "GoogleCloudFinancialservicesV1PredictionResult" + }, + "response": { + "$ref": "Operation" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "Create a PredictionResult." + }, + "patch": { + "id": "financialservices.projects.locations.instances.predictionResults.patch", + "path": "v1/{+name}", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}/instances/{instancesId}/predictionResults/{predictionResultsId}", + "httpMethod": "PATCH", + "parameters": { + "name": { + "description": "Output only. The resource name of the PredictionResult. format: \"/projects/{project_num}/locations/{location}/instances/{instance}/predictionResults/{prediction_result}\"", + "pattern": "^projects/[^/]+/locations/[^/]+/instances/[^/]+/predictionResults/[^/]+$", + "location": "path", + "required": true, + "type": "string" + }, + "updateMask": { + "description": "Optional. Field mask is used to specify the fields to be overwritten in the PredictionResult resource by the update. The fields specified in the update_mask are relative to the resource, not the full request. A field will be overwritten if it is in the mask. If the user does not provide a mask then all fields will be overwritten.", + "location": "query", + "type": "string", + "format": "google-fieldmask" + }, + "requestId": { + "description": "Optional. An optional request ID to identify requests. Specify a unique request ID so that if you must retry your request, the server will know to ignore the request if it has already been completed. The server will guarantee that for at least 60 minutes since the first request. For example, consider a situation where you make an initial request and the request times out. If you make the request again with the same request ID, the server can check if original operation with the same request ID was received, and if so, will ignore the second request. This prevents clients from accidentally creating duplicate commitments. The request ID must be a valid UUID with the exception that zero UUID is not supported (00000000-0000-0000-0000-000000000000).", + "location": "query", + "type": "string" + } + }, + "parameterOrder": [ + "name" + ], + "request": { + "$ref": "GoogleCloudFinancialservicesV1PredictionResult" + }, + "response": { + "$ref": "Operation" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "Updates the parameters of a single PredictionResult." + }, + "exportMetadata": { + "id": "financialservices.projects.locations.instances.predictionResults.exportMetadata", + "path": "v1/{+predictionResult}:exportMetadata", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}/instances/{instancesId}/predictionResults/{predictionResultsId}:exportMetadata", + "httpMethod": "POST", + "parameters": { + "predictionResult": { + "description": "Required. The resource name of the PredictionResult.", + "pattern": "^projects/[^/]+/locations/[^/]+/instances/[^/]+/predictionResults/[^/]+$", + "location": "path", + "required": true, + "type": "string" + } + }, + "parameterOrder": [ + "predictionResult" + ], + "request": { + "$ref": "GoogleCloudFinancialservicesV1ExportPredictionResultMetadataRequest" + }, + "response": { + "$ref": "Operation" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "Export governance information for a PredictionResult resource. For information on the exported fields, see [AML output data model](https://cloud.google.com/financial-services/anti-money-laundering/docs/reference/schemas/aml-output-data-model#prediction-results)." + }, + "delete": { + "id": "financialservices.projects.locations.instances.predictionResults.delete", + "path": "v1/{+name}", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}/instances/{instancesId}/predictionResults/{predictionResultsId}", + "httpMethod": "DELETE", + "parameters": { + "name": { + "description": "Required. The resource name of the PredictionResult.", + "pattern": "^projects/[^/]+/locations/[^/]+/instances/[^/]+/predictionResults/[^/]+$", + "location": "path", + "required": true, + "type": "string" + }, + "requestId": { + "description": "Optional. An optional request ID to identify requests. Specify a unique request ID so that if you must retry your request, the server will know to ignore the request if it has already been completed. The server will guarantee that for at least 60 minutes after the first request. For example, consider a situation where you make an initial request and the request times out. If you make the request again with the same request ID, the server can check if original operation with the same request ID was received, and if so, will ignore the second request. This prevents clients from accidentally creating duplicate commitments. The request ID must be a valid UUID with the exception that zero UUID is not supported (00000000-0000-0000-0000-000000000000).", + "location": "query", + "type": "string" + } + }, + "parameterOrder": [ + "name" + ], + "response": { + "$ref": "Operation" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "Deletes a PredictionResult." + } + } + }, + "backtestResults": { + "methods": { + "list": { + "id": "financialservices.projects.locations.instances.backtestResults.list", + "path": "v1/{+parent}/backtestResults", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}/instances/{instancesId}/backtestResults", + "httpMethod": "GET", + "parameters": { + "parent": { + "description": "Required. The parent of the BacktestResult is the Instance.", + "pattern": "^projects/[^/]+/locations/[^/]+/instances/[^/]+$", + "location": "path", + "required": true, + "type": "string" + }, + "pageSize": { + "description": "The number of resources to be included in the response. The response contains a next_page_token, which can be used to retrieve the next page of resources.", + "location": "query", + "type": "integer", + "format": "int32" + }, + "pageToken": { + "description": "In case of paginated results, this is the token that was returned in the previous ListBacktestResultsResponse. It should be copied here to retrieve the next page of resources. Empty will give the first page of ListBacktestResultsRequest, and the last page will return an empty page_token.", + "location": "query", + "type": "string" + }, + "filter": { + "description": "Specify a filter to narrow search results.", + "location": "query", + "type": "string" + }, + "orderBy": { + "description": "Specify a field to use for ordering.", + "location": "query", + "type": "string" + } + }, + "parameterOrder": [ + "parent" + ], + "response": { + "$ref": "GoogleCloudFinancialservicesV1ListBacktestResultsResponse" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "List BacktestResults." + }, + "get": { + "id": "financialservices.projects.locations.instances.backtestResults.get", + "path": "v1/{+name}", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}/instances/{instancesId}/backtestResults/{backtestResultsId}", + "httpMethod": "GET", + "parameters": { + "name": { + "description": "Required. The resource name of the BacktestResult", + "pattern": "^projects/[^/]+/locations/[^/]+/instances/[^/]+/backtestResults/[^/]+$", + "location": "path", + "required": true, + "type": "string" + } + }, + "parameterOrder": [ + "name" + ], + "response": { + "$ref": "GoogleCloudFinancialservicesV1BacktestResult" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "Gets a BacktestResult." + }, + "create": { + "id": "financialservices.projects.locations.instances.backtestResults.create", + "path": "v1/{+parent}/backtestResults", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}/instances/{instancesId}/backtestResults", + "httpMethod": "POST", + "parameters": { + "parent": { + "description": "Required. The parent of the BacktestResult is the Instance.", + "pattern": "^projects/[^/]+/locations/[^/]+/instances/[^/]+$", + "location": "path", + "required": true, + "type": "string" + }, + "backtestResultId": { + "description": "Required. The resource id of the BacktestResult", + "location": "query", + "type": "string" + }, + "requestId": { + "description": "Optional. An optional request ID to identify requests. Specify a unique request ID so that if you must retry your request, the server will know to ignore the request if it has already been completed. The server will guarantee that for at least 60 minutes since the first request. For example, consider a situation where you make an initial request and the request times out. If you make the request again with the same request ID, the server can check if original operation with the same request ID was received, and if so, will ignore the second request. This prevents clients from accidentally creating duplicate commitments. The request ID must be a valid UUID with the exception that zero UUID is not supported (00000000-0000-0000-0000-000000000000).", + "location": "query", + "type": "string" + } + }, + "parameterOrder": [ + "parent" + ], + "request": { + "$ref": "GoogleCloudFinancialservicesV1BacktestResult" + }, + "response": { + "$ref": "Operation" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "Create a BacktestResult." + }, + "patch": { + "id": "financialservices.projects.locations.instances.backtestResults.patch", + "path": "v1/{+name}", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}/instances/{instancesId}/backtestResults/{backtestResultsId}", + "httpMethod": "PATCH", + "parameters": { + "name": { + "description": "Output only. The resource name of the BacktestResult. format: \"/projects/{project_num}/locations/{location}/instances/{instance}/backtestResults/{backtest_result}\"", + "pattern": "^projects/[^/]+/locations/[^/]+/instances/[^/]+/backtestResults/[^/]+$", + "location": "path", + "required": true, + "type": "string" + }, + "updateMask": { + "description": "Optional. Field mask is used to specify the fields to be overwritten in the BacktestResult resource by the update. The fields specified in the update_mask are relative to the resource, not the full request. A field will be overwritten if it is in the mask. If the user does not provide a mask then all fields will be overwritten.", + "location": "query", + "type": "string", + "format": "google-fieldmask" + }, + "requestId": { + "description": "Optional. An optional request ID to identify requests. Specify a unique request ID so that if you must retry your request, the server will know to ignore the request if it has already been completed. The server will guarantee that for at least 60 minutes since the first request. For example, consider a situation where you make an initial request and the request times out. If you make the request again with the same request ID, the server can check if original operation with the same request ID was received, and if so, will ignore the second request. This prevents clients from accidentally creating duplicate commitments. The request ID must be a valid UUID with the exception that zero UUID is not supported (00000000-0000-0000-0000-000000000000).", + "location": "query", + "type": "string" + } + }, + "parameterOrder": [ + "name" + ], + "request": { + "$ref": "GoogleCloudFinancialservicesV1BacktestResult" + }, + "response": { + "$ref": "Operation" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "Updates the parameters of a single BacktestResult." + }, + "exportMetadata": { + "id": "financialservices.projects.locations.instances.backtestResults.exportMetadata", + "path": "v1/{+backtestResult}:exportMetadata", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}/instances/{instancesId}/backtestResults/{backtestResultsId}:exportMetadata", + "httpMethod": "POST", + "parameters": { + "backtestResult": { + "description": "Required. The resource name of the BacktestResult.", + "pattern": "^projects/[^/]+/locations/[^/]+/instances/[^/]+/backtestResults/[^/]+$", + "location": "path", + "required": true, + "type": "string" + } + }, + "parameterOrder": [ + "backtestResult" + ], + "request": { + "$ref": "GoogleCloudFinancialservicesV1ExportBacktestResultMetadataRequest" + }, + "response": { + "$ref": "Operation" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "Export governance information for a BacktestResult resource. For information on the exported fields, see [AML output data model](https://cloud.google.com/financial-services/anti-money-laundering/docs/reference/schemas/aml-output-data-model#backtest-results)." + }, + "delete": { + "id": "financialservices.projects.locations.instances.backtestResults.delete", + "path": "v1/{+name}", + "flatPath": "v1/projects/{projectsId}/locations/{locationsId}/instances/{instancesId}/backtestResults/{backtestResultsId}", + "httpMethod": "DELETE", + "parameters": { + "name": { + "description": "Required. The resource name of the BacktestResult.", + "pattern": "^projects/[^/]+/locations/[^/]+/instances/[^/]+/backtestResults/[^/]+$", + "location": "path", + "required": true, + "type": "string" + }, + "requestId": { + "description": "Optional. An optional request ID to identify requests. Specify a unique request ID so that if you must retry your request, the server will know to ignore the request if it has already been completed. The server will guarantee that for at least 60 minutes after the first request. For example, consider a situation where you make an initial request and the request times out. If you make the request again with the same request ID, the server can check if original operation with the same request ID was received, and if so, will ignore the second request. This prevents clients from accidentally creating duplicate commitments. The request ID must be a valid UUID with the exception that zero UUID is not supported (00000000-0000-0000-0000-000000000000).", + "location": "query", + "type": "string" + } + }, + "parameterOrder": [ + "name" + ], + "response": { + "$ref": "Operation" + }, + "scopes": [ + "https://www.googleapis.com/auth/cloud-platform" + ], + "description": "Deletes a BacktestResult." + } + } + } + } + } + } + } + } + } + }, + "version_module": true, + "protocol": "rest", + "ownerDomain": "google.com", + "fullyEncodeReservedExpansion": true, + "rootUrl": "https://financialservices.googleapis.com/", + "mtlsRootUrl": "https://financialservices.mtls.googleapis.com/", + "canonicalName": "Financial Services", + "discoveryVersion": "v1", + "batchPath": "batch", + "description": "", + "title": "Financial Services API", + "revision": "20241008" + } diff --git a/providers/tests/google/cloud/hooks/test_financial_services.py b/providers/tests/google/cloud/hooks/test_financial_services.py new file mode 100644 index 000000000000..a6290f2403cd --- /dev/null +++ b/providers/tests/google/cloud/hooks/test_financial_services.py @@ -0,0 +1,120 @@ +# 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 patch + +from airflow.providers.google.cloud.hooks.financial_services import FinancialServicesHook + +TEST_OPERATION = {"name": "test-operation", "metadata": {}, "done": False} + +TEST_INSTANCE = { + "name": "test-instance", + "createTime": "2014-10-02T15:01:23Z", + "updateTime": "2014-10-02T15:01:23Z", + "labels": {}, + "state": "ACTIVE", + "kmsKey": "projects/test-project/locations/us-central1/keyRings/my-kr/cryptoKeys/my-kms-key", +} + + +def mock_init( + self, + gcp_conn_id, + impersonation_chain=None, +): + pass + + +class TestFinancialServicesHook: + def setup_method(self): + with patch( + "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.__init__", + new=mock_init, + ): + self.financial_services_hook = FinancialServicesHook() + + @patch("airflow.providers.google.cloud.hooks.financial_services.build_from_document") + def test_get_conn(self, mock_build): + conn = self.financial_services_hook.get_conn() + + mock_build.assert_called_once() + assert conn == mock_build.return_value + assert conn == self.financial_services_hook.connection + + @patch("airflow.providers.google.cloud.hooks.financial_services.FinancialServicesHook.get_conn") + def test_get_instance(self, mock_get_conn): + self.financial_services_hook.connection = mock_get_conn.return_value + + projects = self.financial_services_hook.connection.projects.return_value + locations = projects.locations.return_value + instances = locations.instances.return_value + instances.get.return_value.execute.return_value = TEST_INSTANCE + + response = self.financial_services_hook.get_instance( + instance_resource_uri="projects/test-project/locations/us-central1/instances/unit-test-instance" + ) + + instances.get.assert_called_once_with( + name="projects/test-project/locations/us-central1/instances/unit-test-instance" + ) + + assert response == TEST_INSTANCE + + @patch("airflow.providers.google.cloud.hooks.financial_services.FinancialServicesHook.get_conn") + def test_create_instance(self, mock_get_conn): + self.financial_services_hook.connection = mock_get_conn.return_value + + projects = self.financial_services_hook.connection.projects.return_value + locations = projects.locations.return_value + instances = locations.instances.return_value + instances.create.return_value.execute.return_value = TEST_OPERATION + + response = self.financial_services_hook.create_instance( + instance_id="test-instance", + kms_key_uri="projects/test-project/locations/us-central1/keyRings/my-kr/cryptoKeys/my-kms-key", + location_resource_uri="projects/test-project/locations/us-central1", + ) + + instances.create.assert_called_once_with( + parent="projects/test-project/locations/us-central1", + instanceId="test-instance", + body={ + "kmsKey": "projects/test-project/locations/us-central1/keyRings/my-kr/cryptoKeys/my-kms-key" + }, + ) + + assert response == TEST_OPERATION + + @patch("airflow.providers.google.cloud.hooks.financial_services.FinancialServicesHook.get_conn") + def test_delete_instance(self, mock_get_conn): + self.financial_services_hook.connection = mock_get_conn.return_value + + projects = self.financial_services_hook.connection.projects.return_value + locations = projects.locations.return_value + instances = locations.instances.return_value + instances.delete.return_value.execute.return_value = TEST_OPERATION + + response = self.financial_services_hook.delete_instance( + instance_resource_uri="projects/test-project/locations/us-central1/instances/unit-test-instance" + ) + + instances.delete.assert_called_once_with( + name="projects/test-project/locations/us-central1/instances/unit-test-instance" + ) + + assert response == TEST_OPERATION From 2994681880f883d234094407cdcf71aae3b5becf Mon Sep 17 00:00:00 2001 From: chris-law-groundtruth Date: Wed, 23 Oct 2024 15:25:32 +0100 Subject: [PATCH 08/28] add operator unit tests for financial services --- .../google/cloud/hooks/financial_services.py | 14 --- .../cloud/operators/financial_services.py | 30 ------- .../operators/test_financial_services.py | 90 +++++++++++++++++++ 3 files changed, 90 insertions(+), 44 deletions(-) create mode 100644 providers/tests/google/cloud/operators/test_financial_services.py diff --git a/providers/src/airflow/providers/google/cloud/hooks/financial_services.py b/providers/src/airflow/providers/google/cloud/hooks/financial_services.py index 8680c302ed8e..3e1408f064e4 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/financial_services.py +++ b/providers/src/airflow/providers/google/cloud/hooks/financial_services.py @@ -30,14 +30,6 @@ class FinancialServicesHook(GoogleBaseHook): :param gcp_conn_id: Identifier of connection to Google Cloud Platform. Defaults to "google_cloud_default". - :param api_version: API version for the Financial Services API. - Defaults to "v1". - :param dev_key_var: Airflow variable name for accessing/saving the - developer key. If key is not provided, secret value will be stored in a - variable with the default name. Defaults to "AMLAI_API_KEY". - :param dev_key_secret_uri: URI for the GCP secret (Secrets - Manager) containing the developer key. Secret will only be accessed if - dev_key_var does not exist. Defaults to None. """ connection: Resource | None = None @@ -45,9 +37,6 @@ class FinancialServicesHook(GoogleBaseHook): def __init__( self, gcp_conn_id: str = "google_cloud_default", - api_version: str = "v1", - dev_key_var: str = "AMLAI_API_KEY", - dev_key_secret_uri: str | None = None, **kwargs, ) -> None: if kwargs.get("delegate_to") is not None: @@ -60,9 +49,6 @@ def __init__( gcp_conn_id=gcp_conn_id, impersonation_chain=None, ) - self.dev_key_var = dev_key_var - self.dev_key_secret_uri = dev_key_secret_uri - self.api_version = api_version def get_conn(self) -> Resource: """ diff --git a/providers/src/airflow/providers/google/cloud/operators/financial_services.py b/providers/src/airflow/providers/google/cloud/operators/financial_services.py index cd04eb904d13..fa6570ebff28 100644 --- a/providers/src/airflow/providers/google/cloud/operators/financial_services.py +++ b/providers/src/airflow/providers/google/cloud/operators/financial_services.py @@ -55,27 +55,17 @@ def __init__( location_resource_uri: str, kms_key_uri: str, gcp_conn_id: str = "google_cloud_default", - api_version: str = "v1", - dev_key_var: str = "AMLAI_API_KEY", - dev_key_secret_uri: str | None = None, **kwargs, ) -> None: super().__init__(**kwargs) self.instance_id = instance_id self.location_resource_uri = location_resource_uri self.gcp_conn_id = gcp_conn_id - self.api_version = api_version - self.dev_key_var = dev_key_var - self.dev_key_secret_uri = dev_key_secret_uri self.kms_key_uri = kms_key_uri def execute(self, context: Context): - super().execute(context) hook = FinancialServicesHook( gcp_conn_id=self.gcp_conn_id, - api_version=self.api_version, - dev_key_var=self.dev_key_var, - dev_key_secret_uri=self.dev_key_secret_uri, ) response = hook.create_instance( instance_id=self.instance_id, @@ -109,25 +99,15 @@ def __init__( self, instance_resource_uri: str, gcp_conn_id: str = "google_cloud_default", - api_version: str = "v1", - dev_key_var: str = "AMLAI_API_KEY", - dev_key_secret_uri: str | None = None, **kwargs, ) -> None: super().__init__(**kwargs) self.instance_resource_uri = instance_resource_uri self.gcp_conn_id = gcp_conn_id - self.api_version = api_version - self.dev_key_var = dev_key_var - self.dev_key_secret_uri = dev_key_secret_uri def execute(self, context: Context): - super().execute(context) hook = FinancialServicesHook( gcp_conn_id=self.gcp_conn_id, - api_version=self.api_version, - dev_key_var=self.dev_key_var, - dev_key_secret_uri=self.dev_key_secret_uri, ) response = hook.delete_instance( instance_resource_uri=self.instance_resource_uri, @@ -159,25 +139,15 @@ def __init__( self, instance_resource_uri: str, gcp_conn_id: str = "google_cloud_default", - api_version: str = "v1", - dev_key_var: str = "AMLAI_API_KEY", - dev_key_secret_uri: str | None = None, **kwargs, ) -> None: super().__init__(**kwargs) self.instance_resource_uri = instance_resource_uri self.gcp_conn_id = gcp_conn_id - self.api_version = api_version - self.dev_key_var = dev_key_var - self.dev_key_secret_uri = dev_key_secret_uri def execute(self, context: Context): - super().execute(context) hook = FinancialServicesHook( gcp_conn_id=self.gcp_conn_id, - api_version=self.api_version, - dev_key_var=self.dev_key_var, - dev_key_secret_uri=self.dev_key_secret_uri, ) response = hook.get_instance( instance_resource_uri=self.instance_resource_uri, diff --git a/providers/tests/google/cloud/operators/test_financial_services.py b/providers/tests/google/cloud/operators/test_financial_services.py new file mode 100644 index 000000000000..76b29fe1ddcc --- /dev/null +++ b/providers/tests/google/cloud/operators/test_financial_services.py @@ -0,0 +1,90 @@ +# 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 import mock + +from airflow.providers.google.cloud.operators.financial_services import ( + FinancialServicesCreateInstanceOperator, + FinancialServicesDeleteInstanceOperator, +) + +TEST_OPERATION = {"name": "test-operation", "metadata": {}, "done": False} + +TEST_INSTANCE = { + "name": "test-instance", + "createTime": "2014-10-02T15:01:23Z", + "updateTime": "2014-10-02T15:01:23Z", + "labels": {}, + "state": "ACTIVE", + "kmsKey": "projects/test-project/locations/us-central1/keyRings/my-kr/cryptoKeys/my-kms-key", +} + + +class TestFinancialServicesCreateInstanceOperator: + @mock.patch("airflow.providers.google.cloud.operators.financial_services.FinancialServicesHook") + def test_execute(self, mock_hook): + mock_hook.return_value.create_instance.return_value = TEST_OPERATION + + op = FinancialServicesCreateInstanceOperator( + task_id="test_create_instance_task", + instance_id="test-instance", + kms_key_uri="projects/test-project/locations/us-central1/keyRings/my-kr/cryptoKeys/my-kms-key", + location_resource_uri="projects/test-project/locations/us-central1", + ) + op.execute(context={"ti": mock.MagicMock()}) + + mock_hook.assert_called_once_with(gcp_conn_id="google_cloud_default") + mock_hook.return_value.create_instance.assert_called_once_with( + instance_id="test-instance", + kms_key_uri="projects/test-project/locations/us-central1/keyRings/my-kr/cryptoKeys/my-kms-key", + location_resource_uri="projects/test-project/locations/us-central1", + ) + + +class TestFinancialServicesDeleteInstanceOperator: + @mock.patch("airflow.providers.google.cloud.operators.financial_services.FinancialServicesHook") + def test_execute(self, mock_hook): + mock_hook.return_value.delete_instance.return_value = TEST_OPERATION + + op = FinancialServicesDeleteInstanceOperator( + task_id="test_delete_instance_task", + instance_resource_uri="projects/test-project/locations/us-central1/instances/test-instance", + ) + op.execute(context={"ti": mock.MagicMock()}) + + mock_hook.assert_called_once_with(gcp_conn_id="google_cloud_default") + mock_hook.return_value.delete_instance.assert_called_once_with( + instance_resource_uri="projects/test-project/locations/us-central1/instances/test-instance" + ) + + +class TestFinancialServicesGetInstanceOperator: + @mock.patch("airflow.providers.google.cloud.operators.financial_services.FinancialServicesHook") + def test_execute(self, mock_hook): + mock_hook.return_value.get_instance.return_value = TEST_INSTANCE + + op = FinancialServicesDeleteInstanceOperator( + task_id="test_get_instance_task", + instance_resource_uri="projects/test-project/locations/us-central1/instances/test-instance", + ) + op.execute(context={"ti": mock.MagicMock()}) + + mock_hook.assert_called_once_with(gcp_conn_id="google_cloud_default") + mock_hook.return_value.delete_instance.assert_called_once_with( + instance_resource_uri="projects/test-project/locations/us-central1/instances/test-instance" + ) From 3ea98919ad5d9978d889904dfb3565cf82048779 Mon Sep 17 00:00:00 2001 From: chris-law-groundtruth Date: Wed, 23 Oct 2024 15:51:34 +0100 Subject: [PATCH 09/28] add sensor unit tests for financial services --- .../cloud/sensors/financial_services.py | 10 --- .../cloud/sensors/test_financial_services.py | 78 +++++++++++++++++++ 2 files changed, 78 insertions(+), 10 deletions(-) create mode 100644 providers/tests/google/cloud/sensors/test_financial_services.py diff --git a/providers/src/airflow/providers/google/cloud/sensors/financial_services.py b/providers/src/airflow/providers/google/cloud/sensors/financial_services.py index 5f5838af029e..ca92db729f7a 100644 --- a/providers/src/airflow/providers/google/cloud/sensors/financial_services.py +++ b/providers/src/airflow/providers/google/cloud/sensors/financial_services.py @@ -50,25 +50,15 @@ def __init__( self, operation_resource_uri: str, gcp_conn_id: str = "google_cloud_default", - api_version: str = "v1", - dev_key_var: str = "AMLAI_API_KEY", - dev_key_secret_uri: str | None = None, **kwargs, ): super().__init__(**kwargs) self.operation_resource_uri = operation_resource_uri self.gcp_conn_id = gcp_conn_id - self.api_version = api_version - self.dev_key_var = dev_key_var - self.dev_key_secret_uri = dev_key_secret_uri def poke(self, context: Context) -> PokeReturnValue: - super().poke(context) hook = FinancialServicesHook( gcp_conn_id=self.gcp_conn_id, - api_version=self.api_version, - dev_key_var=self.dev_key_var, - dev_key_secret_uri=self.dev_key_secret_uri, ) operation = hook.get_operation(operation_resource_uri=self.operation_resource_uri) if "error" in operation.keys(): diff --git a/providers/tests/google/cloud/sensors/test_financial_services.py b/providers/tests/google/cloud/sensors/test_financial_services.py new file mode 100644 index 000000000000..b64632e55414 --- /dev/null +++ b/providers/tests/google/cloud/sensors/test_financial_services.py @@ -0,0 +1,78 @@ +# 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 import mock + +import pytest + +from airflow.exceptions import AirflowFailException +from airflow.providers.google.cloud.sensors.financial_services import FinancialServicesOperationSensor + +TEST_OPERATION = {"name": "test-operation", "metadata": {}, "done": False} +TEST_OPERATION_DONE = {"name": "test-operation", "metadata": {}, "done": True, "response": {}} +TEST_OPERATION_ERROR = {"name": "test-operation", "metadata": {}, "done": True, "error": {}} + + +class TestFinancialServicesOperationSensor: + @mock.patch("airflow.providers.google.cloud.sensors.financial_services.FinancialServicesHook") + def test_poke(self, mock_hook): + mock_hook.return_value.get_operation.return_value = TEST_OPERATION + + op = FinancialServicesOperationSensor( + task_id="test_operation_sensor_task", + operation_resource_uri="projects/test-project/locations/us-central1/instances/test-instance", + ) + response = op.poke(context={"ti": mock.MagicMock()}) + + mock_hook.assert_called_once_with(gcp_conn_id="google_cloud_default") + mock_hook.return_value.get_operation.assert_called_once_with( + operation_resource_uri="projects/test-project/locations/us-central1/instances/test-instance", + ) + assert response.is_done == TEST_OPERATION["done"] + + @mock.patch("airflow.providers.google.cloud.sensors.financial_services.FinancialServicesHook") + def test_poke_done(self, mock_hook): + mock_hook.return_value.get_operation.return_value = TEST_OPERATION_DONE + + op = FinancialServicesOperationSensor( + task_id="test_operation_sensor_task", + operation_resource_uri="projects/test-project/locations/us-central1/instances/test-instance", + ) + response = op.poke(context={"ti": mock.MagicMock()}) + + mock_hook.assert_called_once_with(gcp_conn_id="google_cloud_default") + mock_hook.return_value.get_operation.assert_called_once_with( + operation_resource_uri="projects/test-project/locations/us-central1/instances/test-instance", + ) + assert response.is_done == TEST_OPERATION_DONE["done"] + + @mock.patch("airflow.providers.google.cloud.sensors.financial_services.FinancialServicesHook") + def test_poke_error(self, mock_hook): + mock_hook.return_value.get_operation.return_value = TEST_OPERATION_ERROR + + op = FinancialServicesOperationSensor( + task_id="test_operation_sensor_task", + operation_resource_uri="projects/test-project/locations/us-central1/instances/test-instance", + ) + with pytest.raises(AirflowFailException): + op.poke(context={"ti": mock.MagicMock()}) + + mock_hook.assert_called_once_with(gcp_conn_id="google_cloud_default") + mock_hook.return_value.get_operation.assert_called_once_with( + operation_resource_uri="projects/test-project/locations/us-central1/instances/test-instance", + ) From 8eb9a8a0f1b472baf1cf11312053502089aca992 Mon Sep 17 00:00:00 2001 From: chris-law-groundtruth Date: Wed, 23 Oct 2024 15:54:14 +0100 Subject: [PATCH 10/28] update docstrings by removing args related to googleapi discovery --- .../google/cloud/hooks/financial_services.py | 2 +- .../cloud/operators/financial_services.py | 24 ------------------- .../cloud/sensors/financial_services.py | 8 ------- 3 files changed, 1 insertion(+), 33 deletions(-) diff --git a/providers/src/airflow/providers/google/cloud/hooks/financial_services.py b/providers/src/airflow/providers/google/cloud/hooks/financial_services.py index 3e1408f064e4..ed4c6b926508 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/financial_services.py +++ b/providers/src/airflow/providers/google/cloud/hooks/financial_services.py @@ -83,7 +83,7 @@ def create_instance(self, instance_id: str, kms_key_uri: str, location_resource_ Create an AML AI instance. :param instance_id: Identifier for the instance to create - :param kms_key: URI of the KMS key to that will be used for instance encryption + :param kms_key_uri: URI of the KMS key to that will be used for instance encryption (format: 'projects//locations//keyRings// cryptoKeys/') diff --git a/providers/src/airflow/providers/google/cloud/operators/financial_services.py b/providers/src/airflow/providers/google/cloud/operators/financial_services.py index fa6570ebff28..76880e4a8d3a 100644 --- a/providers/src/airflow/providers/google/cloud/operators/financial_services.py +++ b/providers/src/airflow/providers/google/cloud/operators/financial_services.py @@ -37,14 +37,6 @@ class FinancialServicesCreateInstanceOperator(GoogleCloudBaseOperator): /cryptoKeys/') :param gcp_conn_id: Identifier of connection to Google Cloud Platform. Defaults to "google_cloud_default". - :param api_version: API version for the Financial Services API. - Defaults to "v1". - :param dev_key_var: Airflow variable name for accessing/saving the - developer key. If key is not provided, secret value will be stored in a - variable with the default name. Defaults to "AMLAI_API_KEY". - :param dev_key_secret_uri: URI for the GCP secret (Secrets - Manager) containing the developer key. Secret will only be accessed if - dev_key_var does not exist. Defaults to None. """ template_fields: Sequence[str] = ("instance_id", "location_resource_uri", "kms_key_uri") @@ -83,14 +75,6 @@ class FinancialServicesDeleteInstanceOperator(GoogleCloudBaseOperator): 'projects//locations//instances/) :param gcp_conn_id: Identifier of connection to Google Cloud Platform. Defaults to "google_cloud_default". - :param api_version: API version for the Financial Services API. - Defaults to "v1". - :param dev_key_var: Airflow variable name for accessing/saving the - developer key. If key is not provided, secret value will be stored in a - variable with the default name. Defaults to "AMLAI_API_KEY". - :param dev_key_secret_uri: URI for the GCP secret (Secrets - Manager) containing the developer key. Secret will only be accessed if - dev_key_var does not exist. Defaults to None. """ template_fields: Sequence[str] = "instance_resource_uri" @@ -123,14 +107,6 @@ class FinancialServicesGetInstanceOperator(GoogleCloudBaseOperator): 'projects//locations//instances/) :param gcp_conn_id: Identifier of connection to Google Cloud Platform. Defaults to "google_cloud_default". - :param api_version: API version for the Financial Services API. - Defaults to "v1". - :param dev_key_var: Airflow variable name for accessing/saving the - developer key. If key is not provided, secret value will be stored in a - variable with the default name. Defaults to "AMLAI_API_KEY". - :param dev_key_secret_uri: URI for the GCP secret (Secrets - Manager) containing the developer key. Secret will only be accessed if - dev_key_var does not exist. Defaults to None. """ template_fields: Sequence[str] = "instance_resource_uri" diff --git a/providers/src/airflow/providers/google/cloud/sensors/financial_services.py b/providers/src/airflow/providers/google/cloud/sensors/financial_services.py index ca92db729f7a..a2c9ecf8372f 100644 --- a/providers/src/airflow/providers/google/cloud/sensors/financial_services.py +++ b/providers/src/airflow/providers/google/cloud/sensors/financial_services.py @@ -34,14 +34,6 @@ class FinancialServicesOperationSensor(BaseSensorOperator): 'projects//locations//operations/) :param gcp_conn_id: Identifier of connection to Google Cloud Platform. Defaults to "google_cloud_default". - :param api_version: API version for the Financial Services API. - Defaults to "v1". - :param dev_key_var: Airflow variable name for accessing/saving the - developer key. If key is not provided, secret value will be stored in a - variable with the default name. Defaults to "AMLAI_API_KEY". - :param dev_key_secret_uri: URI for the GCP secret (Secrets - Manager) containing the developer key. Secret will only be accessed if - dev_key_var does not exist. Defaults to None. """ template_fields: Sequence[str] = ("operation_resource_uri",) From 18215a6f9970904d4f0cb93105fbb9fc45fc0841 Mon Sep 17 00:00:00 2001 From: chris-law-groundtruth Date: Thu, 24 Oct 2024 12:04:59 +0100 Subject: [PATCH 11/28] add system test for financial services --- .../cloud/operators/financial_services.py | 4 +- .../cloud/financial_services/__init__.py | 16 ++++ .../example_financial_services.py | 95 +++++++++++++++++++ .../financial_services/resources/__init__.py | 16 ++++ 4 files changed, 129 insertions(+), 2 deletions(-) create mode 100644 providers/tests/system/google/cloud/financial_services/__init__.py create mode 100644 providers/tests/system/google/cloud/financial_services/example_financial_services.py create mode 100644 providers/tests/system/google/cloud/financial_services/resources/__init__.py diff --git a/providers/src/airflow/providers/google/cloud/operators/financial_services.py b/providers/src/airflow/providers/google/cloud/operators/financial_services.py index 76880e4a8d3a..66041df87b57 100644 --- a/providers/src/airflow/providers/google/cloud/operators/financial_services.py +++ b/providers/src/airflow/providers/google/cloud/operators/financial_services.py @@ -77,7 +77,7 @@ class FinancialServicesDeleteInstanceOperator(GoogleCloudBaseOperator): Defaults to "google_cloud_default". """ - template_fields: Sequence[str] = "instance_resource_uri" + template_fields: Sequence[str] = ("instance_resource_uri",) def __init__( self, @@ -109,7 +109,7 @@ class FinancialServicesGetInstanceOperator(GoogleCloudBaseOperator): Defaults to "google_cloud_default". """ - template_fields: Sequence[str] = "instance_resource_uri" + template_fields: Sequence[str] = ("instance_resource_uri",) def __init__( self, diff --git a/providers/tests/system/google/cloud/financial_services/__init__.py b/providers/tests/system/google/cloud/financial_services/__init__.py new file mode 100644 index 000000000000..13a83393a912 --- /dev/null +++ b/providers/tests/system/google/cloud/financial_services/__init__.py @@ -0,0 +1,16 @@ +# 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. diff --git a/providers/tests/system/google/cloud/financial_services/example_financial_services.py b/providers/tests/system/google/cloud/financial_services/example_financial_services.py new file mode 100644 index 000000000000..9d30bdf3adfd --- /dev/null +++ b/providers/tests/system/google/cloud/financial_services/example_financial_services.py @@ -0,0 +1,95 @@ +# 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 os +from datetime import datetime, timedelta + +from airflow.models.dag import DAG +from airflow.providers.google.cloud.operators.financial_services import ( + FinancialServicesCreateInstanceOperator, + FinancialServicesDeleteInstanceOperator, + FinancialServicesGetInstanceOperator, +) +from airflow.providers.google.cloud.sensors.financial_services import FinancialServicesOperationSensor + +from providers.tests.system.google import DEFAULT_GCP_SYSTEM_TEST_PROJECT_ID + +ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID", "default") +PROJECT_ID = os.environ.get("SYSTEM_TESTS_GCP_PROJECT") or DEFAULT_GCP_SYSTEM_TEST_PROJECT_ID +LOCATION = os.environ.get("SYSTEM_TESTS_GCP_LOCATION", "us-central1") +KMS_KEY = os.environ.get("SYSTEM_TESTS_GCP_KMS_KEY") + +DAG_ID = "financial_services_instance" + +INSTANCE_ID = f"instance_{DAG_ID}_{ENV_ID}" + +with DAG( + DAG_ID, + schedule="@once", + start_date=datetime(2021, 1, 1), + catchup=False, +) as dag: + create_instance_task = FinancialServicesCreateInstanceOperator( + task_id="create_instance_task", + instance_id=INSTANCE_ID, + location_resource_uri=f"projects/{PROJECT_ID}/locations/{LOCATION}", + kms_key_uri=KMS_KEY, + ) + + create_instance_sensor = FinancialServicesOperationSensor( + task_id="create_instance_sensor", + operation_resource_uri="{{ task_instance.xcom_pull(task_ids='create_instance_task', key='return_value') }}", + poke_interval=timedelta(seconds=5), + timeout=timedelta(hours=1), + ) + + get_instance_task = FinancialServicesGetInstanceOperator( + task_id="get_instance_task", + instance_resource_uri=f"projects/{PROJECT_ID}/locations/{LOCATION}/instances/{INSTANCE_ID}", + ) + + delete_instance_task = FinancialServicesDeleteInstanceOperator( + task_id="delete_instance_task", + instance_resource_uri=f"projects/{PROJECT_ID}/locations/{LOCATION}/instances/{INSTANCE_ID}", + ) + + delete_instance_sensor = FinancialServicesOperationSensor( + task_id="delete_instance_sensor", + operation_resource_uri="{{ task_instance.xcom_pull(task_ids='delete_instance_task', key='return_value') }}", + poke_interval=timedelta(seconds=5), + timeout=timedelta(hours=1), + ) + + ( + create_instance_task + >> create_instance_sensor + >> get_instance_task + >> delete_instance_task + >> delete_instance_sensor + ) + + from tests_common.test_utils.watcher import watcher + + # This test needs watcher in order to properly mark success/failure + # when "teardown" task with trigger rule is part of the DAG + list(dag.tasks) >> watcher() + +from tests_common.test_utils.system_tests import get_test_run # noqa: E402 + +# Needed to run the example DAG with pytest (see: tests/system/README.md#run_via_pytest) +test_run = get_test_run(dag) diff --git a/providers/tests/system/google/cloud/financial_services/resources/__init__.py b/providers/tests/system/google/cloud/financial_services/resources/__init__.py new file mode 100644 index 000000000000..13a83393a912 --- /dev/null +++ b/providers/tests/system/google/cloud/financial_services/resources/__init__.py @@ -0,0 +1,16 @@ +# 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 fb6adda06b54a16790cd828be8a338d11617d36f Mon Sep 17 00:00:00 2001 From: chris-law-groundtruth Date: Thu, 24 Oct 2024 12:42:53 +0100 Subject: [PATCH 12/28] add docs for FinancialServicesCreateInstanceOperator --- .../operators/cloud/financial_services.rst | 62 +++++++++++++++++++ .../cloud/operators/financial_services.py | 2 + .../airflow/providers/google/provider.yaml | 14 +++++ .../example_financial_services.py | 2 + 4 files changed, 80 insertions(+) create mode 100644 docs/apache-airflow-providers-google/operators/cloud/financial_services.rst diff --git a/docs/apache-airflow-providers-google/operators/cloud/financial_services.rst b/docs/apache-airflow-providers-google/operators/cloud/financial_services.rst new file mode 100644 index 000000000000..ad3951b394bc --- /dev/null +++ b/docs/apache-airflow-providers-google/operators/cloud/financial_services.rst @@ -0,0 +1,62 @@ + .. 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. + + + +Google Cloud Functions Operators +================================ + +Prerequisite Tasks +^^^^^^^^^^^^^^^^^^ + +.. include:: /operators/_partials/prerequisite_tasks.rst + +.. _howto/operator:FinancialServicesCreateInstanceOperator: + +FinancialServicesCreateInstanceOperator +--------------------------------------- + +Use the operator to create an AML AI instance in the Google Cloud Financial Services API. + +For parameter definition, take a look at +:class:`~airflow.providers.google.cloud.operators.financial_services.FinancialServicesCreateInstanceOperator`. + +Using the operator +"""""""""""""""""" + +.. exampleinclude:: /../../providers/tests/system/google/cloud/financial_services/example_financial_services.py + :language: python + :dedent: 4 + :start-after: [START howto_operator_financial_services_create_instance] + :end-before: [END howto_operator_financial_services_create_instance] + +Templating +"""""""""" + +.. literalinclude:: /../../providers/src/airflow/providers/google/cloud/operators/financial_services.py + :language: python + :dedent: 4 + :start-after: [START howto_operator_financial_services_create_instance_template_fields] + :end-before: [END howto_operator_financial_services_create_instance_template_fields] + +More information +"""""""""""""""" + +See Google Cloud Financial Services API documentation to `create an instance +`_. + +.. _howto/operator:FinancialServicesCreateInstanceOperator: diff --git a/providers/src/airflow/providers/google/cloud/operators/financial_services.py b/providers/src/airflow/providers/google/cloud/operators/financial_services.py index 66041df87b57..82bb69a8dfc1 100644 --- a/providers/src/airflow/providers/google/cloud/operators/financial_services.py +++ b/providers/src/airflow/providers/google/cloud/operators/financial_services.py @@ -39,7 +39,9 @@ class FinancialServicesCreateInstanceOperator(GoogleCloudBaseOperator): Defaults to "google_cloud_default". """ + # [START howto_operator_financial_services_create_instance_template_fields] template_fields: Sequence[str] = ("instance_id", "location_resource_uri", "kms_key_uri") + # [END howto_operator_financial_services_create_instance_template_fields] def __init__( self, diff --git a/providers/src/airflow/providers/google/provider.yaml b/providers/src/airflow/providers/google/provider.yaml index 09e7b6643cb2..b18b632fcd47 100644 --- a/providers/src/airflow/providers/google/provider.yaml +++ b/providers/src/airflow/providers/google/provider.yaml @@ -284,6 +284,11 @@ integrations: - /docs/apache-airflow-providers-google/operators/firebase/firestore.rst logo: /integration-logos/gcp/Google-Firestore.png tags: [gcp] + - integration-name: Google Cloud Financial Services + external-doc-url: https://cloud.google.com/financial-services/anti-money-laundering/docs + how-to-guide: + - /docs/apache-airflow-providers-google/operators/cloud/financial_services.rst + tags: [gcp] - integration-name: Google Cloud Functions external-doc-url: https://cloud.google.com/functions/ how-to-guide: @@ -613,6 +618,9 @@ operators: - integration-name: Google Cloud Data Loss Prevention (DLP) python-modules: - airflow.providers.google.cloud.operators.dlp + - integration-name: Google Cloud Financial Services + python-modules: + - airflow.providers.google.cloud.operators.financial_services - integration-name: Google Cloud Functions python-modules: - airflow.providers.google.cloud.operators.functions @@ -742,6 +750,9 @@ sensors: - integration-name: Google Dataproc Metastore python-modules: - airflow.providers.google.cloud.sensors.dataproc_metastore + - integration-name: Google Cloud Financial Services + python-modules: + - airflow.providers.google.cloud.sensors.financial_services - integration-name: Google Cloud Storage (GCS) python-modules: - airflow.providers.google.cloud.sensors.gcs @@ -855,6 +866,9 @@ hooks: - integration-name: Google Cloud Data Loss Prevention (DLP) python-modules: - airflow.providers.google.cloud.hooks.dlp + - integration-name: Google Cloud Financial Services + python-modules: + - airflow.providers.google.cloud.hooks.financial_services - integration-name: Google Cloud Functions python-modules: - airflow.providers.google.cloud.hooks.functions diff --git a/providers/tests/system/google/cloud/financial_services/example_financial_services.py b/providers/tests/system/google/cloud/financial_services/example_financial_services.py index 9d30bdf3adfd..65195b1211f4 100644 --- a/providers/tests/system/google/cloud/financial_services/example_financial_services.py +++ b/providers/tests/system/google/cloud/financial_services/example_financial_services.py @@ -44,12 +44,14 @@ start_date=datetime(2021, 1, 1), catchup=False, ) as dag: + # [START howto_operator_financial_services_create_instance] create_instance_task = FinancialServicesCreateInstanceOperator( task_id="create_instance_task", instance_id=INSTANCE_ID, location_resource_uri=f"projects/{PROJECT_ID}/locations/{LOCATION}", kms_key_uri=KMS_KEY, ) + # [END howto_operator_financial_services_create_instance] create_instance_sensor = FinancialServicesOperationSensor( task_id="create_instance_sensor", From bdf714023126739bb73d3209ed6a0382367bd243 Mon Sep 17 00:00:00 2001 From: chris-law-groundtruth Date: Thu, 24 Oct 2024 15:11:59 +0100 Subject: [PATCH 13/28] add remaining instance operator docs --- .../operators/cloud/financial_services.rst | 87 +++++++++++++++++-- .../cloud/operators/financial_services.py | 4 + .../example_financial_services.py | 4 + 3 files changed, 87 insertions(+), 8 deletions(-) diff --git a/docs/apache-airflow-providers-google/operators/cloud/financial_services.rst b/docs/apache-airflow-providers-google/operators/cloud/financial_services.rst index ad3951b394bc..603a06c56487 100644 --- a/docs/apache-airflow-providers-google/operators/cloud/financial_services.rst +++ b/docs/apache-airflow-providers-google/operators/cloud/financial_services.rst @@ -17,23 +17,24 @@ -Google Cloud Functions Operators -================================ +Google Cloud Financial Services Operators +========================================= Prerequisite Tasks ^^^^^^^^^^^^^^^^^^ .. include:: /operators/_partials/prerequisite_tasks.rst -.. _howto/operator:FinancialServicesCreateInstanceOperator: +Manage instances +^^^^^^^^^^^^^^^ -FinancialServicesCreateInstanceOperator ---------------------------------------- +.. _howto/operator:FinancialServicesCreateInstanceOperator: -Use the operator to create an AML AI instance in the Google Cloud Financial Services API. +Create an AML AI instance +------------------------- -For parameter definition, take a look at -:class:`~airflow.providers.google.cloud.operators.financial_services.FinancialServicesCreateInstanceOperator`. +Use the :class:`~airflow.providers.google.cloud.operators.financial_services.FinancialServicesCreateInstanceOperator` +operator to create an AML AI instance in the Google Cloud Financial Services API. Using the operator """""""""""""""""" @@ -60,3 +61,73 @@ See Google Cloud Financial Services API documentation to `create an instance `_. .. _howto/operator:FinancialServicesCreateInstanceOperator: + + +.. _howto/operator:FinancialServicesGetInstanceOperator: + +Get an AML AI instance +------------------------- + +Use the :class:`~airflow.providers.google.cloud.operators.financial_services.FinancialServicesGetInstanceOperator` +operator to get an AML AI instance in the Google Cloud Financial Services API. + +Using the operator +"""""""""""""""""" + +.. exampleinclude:: /../../providers/tests/system/google/cloud/financial_services/example_financial_services.py + :language: python + :dedent: 4 + :start-after: [START howto_operator_financial_services_get_instance] + :end-before: [END howto_operator_financial_services_get_instance] + +Templating +"""""""""" + +.. literalinclude:: /../../providers/src/airflow/providers/google/cloud/operators/financial_services.py + :language: python + :dedent: 4 + :start-after: [START howto_operator_financial_services_get_instance_template_fields] + :end-before: [END howto_operator_financial_services_get_instance_template_fields] + +More information +"""""""""""""""" + +See Google Cloud Financial Services API documentation to `get an instance +`_. + +.. _howto/operator:FinancialServicesGetInstanceOperator: + + +.. _howto/operator:FinancialServicesDeleteInstanceOperator: + +Get an AML AI instance +------------------------- + +Use the :class:`~airflow.providers.google.cloud.operators.financial_services.FinancialServicesDeleteInstanceOperator` +operator to delete an AML AI instance in the Google Cloud Financial Services API. + +Using the operator +"""""""""""""""""" + +.. exampleinclude:: /../../providers/tests/system/google/cloud/financial_services/example_financial_services.py + :language: python + :dedent: 4 + :start-after: [START howto_operator_financial_services_delete_instance] + :end-before: [END howto_operator_financial_services_delete_instance] + +Templating +"""""""""" + +.. literalinclude:: /../../providers/src/airflow/providers/google/cloud/operators/financial_services.py + :language: python + :dedent: 4 + :start-after: [START howto_operator_financial_services_delete_instance_template_fields] + :end-before: [END howto_operator_financial_services_delete_instance_template_fields] + +More information +"""""""""""""""" + +See Google Cloud Financial Services API documentation to `delete an instance +`_. + +.. _howto/operator:FinancialServicesDeleteInstanceOperator: diff --git a/providers/src/airflow/providers/google/cloud/operators/financial_services.py b/providers/src/airflow/providers/google/cloud/operators/financial_services.py index 82bb69a8dfc1..1ae91df36f46 100644 --- a/providers/src/airflow/providers/google/cloud/operators/financial_services.py +++ b/providers/src/airflow/providers/google/cloud/operators/financial_services.py @@ -79,7 +79,9 @@ class FinancialServicesDeleteInstanceOperator(GoogleCloudBaseOperator): Defaults to "google_cloud_default". """ + # [START howto_operator_financial_services_get_instance_template_fields] template_fields: Sequence[str] = ("instance_resource_uri",) + # [END howto_operator_financial_services_get_instance_template_fields] def __init__( self, @@ -111,7 +113,9 @@ class FinancialServicesGetInstanceOperator(GoogleCloudBaseOperator): Defaults to "google_cloud_default". """ + # [START howto_operator_financial_services_delete_instance_template_fields] template_fields: Sequence[str] = ("instance_resource_uri",) + # [END howto_operator_financial_services_delete_instance_template_fields] def __init__( self, diff --git a/providers/tests/system/google/cloud/financial_services/example_financial_services.py b/providers/tests/system/google/cloud/financial_services/example_financial_services.py index 65195b1211f4..520bffb1a579 100644 --- a/providers/tests/system/google/cloud/financial_services/example_financial_services.py +++ b/providers/tests/system/google/cloud/financial_services/example_financial_services.py @@ -60,15 +60,19 @@ timeout=timedelta(hours=1), ) + # [START howto_operator_financial_services_get_instance] get_instance_task = FinancialServicesGetInstanceOperator( task_id="get_instance_task", instance_resource_uri=f"projects/{PROJECT_ID}/locations/{LOCATION}/instances/{INSTANCE_ID}", ) + # [END howto_operator_financial_services_get_instance] + # [START howto_operator_financial_services_delete_instance] delete_instance_task = FinancialServicesDeleteInstanceOperator( task_id="delete_instance_task", instance_resource_uri=f"projects/{PROJECT_ID}/locations/{LOCATION}/instances/{INSTANCE_ID}", ) + # [END howto_operator_financial_services_delete_instance] delete_instance_sensor = FinancialServicesOperationSensor( task_id="delete_instance_sensor", From 5027afe1d93cda30bbd0581080ba88506497eb98 Mon Sep 17 00:00:00 2001 From: chris-law-groundtruth Date: Thu, 24 Oct 2024 15:25:32 +0100 Subject: [PATCH 14/28] add operation sensor docs --- .../operators/cloud/financial_services.rst | 42 ++++++++++++++++++- .../cloud/sensors/financial_services.py | 2 + .../example_financial_services.py | 2 + 3 files changed, 44 insertions(+), 2 deletions(-) diff --git a/docs/apache-airflow-providers-google/operators/cloud/financial_services.rst b/docs/apache-airflow-providers-google/operators/cloud/financial_services.rst index 603a06c56487..e51ec04a9d02 100644 --- a/docs/apache-airflow-providers-google/operators/cloud/financial_services.rst +++ b/docs/apache-airflow-providers-google/operators/cloud/financial_services.rst @@ -26,7 +26,7 @@ Prerequisite Tasks .. include:: /operators/_partials/prerequisite_tasks.rst Manage instances -^^^^^^^^^^^^^^^ +^^^^^^^^^^^^^^^^ .. _howto/operator:FinancialServicesCreateInstanceOperator: @@ -100,7 +100,7 @@ See Google Cloud Financial Services API documentation to `get an instance .. _howto/operator:FinancialServicesDeleteInstanceOperator: -Get an AML AI instance +Delete an AML AI instance ------------------------- Use the :class:`~airflow.providers.google.cloud.operators.financial_services.FinancialServicesDeleteInstanceOperator` @@ -131,3 +131,41 @@ See Google Cloud Financial Services API documentation to `delete an instance `_. .. _howto/operator:FinancialServicesDeleteInstanceOperator: + + +Sensors +^^^^^^^ + +.. _howto/sensor:FinancialServicesOperationSensor: + +Check operation status +---------------------- + +Use the :class:`~airflow.providers.google.cloud.sensors.financial_services.FinancialServicesOperationSensor` +sensor to check the status of an operation in the Google Cloud Financial Services API. + +Using the sensor +"""""""""""""""""" + +.. exampleinclude:: /../../providers/tests/system/google/cloud/financial_services/example_financial_services.py + :language: python + :dedent: 4 + :start-after: [START howto_sensor_financial_services_operation] + :end-before: [END howto_sensor_financial_services_operation] + +Templating +"""""""""" + +.. literalinclude:: /../../providers/src/airflow/providers/google/cloud/operators/financial_services.py + :language: python + :dedent: 4 + :start-after: [START howto_sensor_financial_services_operation_template_fields] + :end-before: [END howto_sensor_financial_services_operation_template_fields] + +More information +"""""""""""""""" + +See Google Cloud Financial Services API documentation to `get an operation +`_. + +.. _howto/sensor:FinancialServicesOperationSensor: diff --git a/providers/src/airflow/providers/google/cloud/sensors/financial_services.py b/providers/src/airflow/providers/google/cloud/sensors/financial_services.py index a2c9ecf8372f..a6dcc2f7a1c1 100644 --- a/providers/src/airflow/providers/google/cloud/sensors/financial_services.py +++ b/providers/src/airflow/providers/google/cloud/sensors/financial_services.py @@ -36,7 +36,9 @@ class FinancialServicesOperationSensor(BaseSensorOperator): Defaults to "google_cloud_default". """ + # [START howto_sensor_financial_services_operation_template_fields] template_fields: Sequence[str] = ("operation_resource_uri",) + # [END howto_sensor_financial_services_operation_template_fields] def __init__( self, diff --git a/providers/tests/system/google/cloud/financial_services/example_financial_services.py b/providers/tests/system/google/cloud/financial_services/example_financial_services.py index 520bffb1a579..0db28a6f9e3b 100644 --- a/providers/tests/system/google/cloud/financial_services/example_financial_services.py +++ b/providers/tests/system/google/cloud/financial_services/example_financial_services.py @@ -53,12 +53,14 @@ ) # [END howto_operator_financial_services_create_instance] + # [START howto_sensor_financial_services_operation] create_instance_sensor = FinancialServicesOperationSensor( task_id="create_instance_sensor", operation_resource_uri="{{ task_instance.xcom_pull(task_ids='create_instance_task', key='return_value') }}", poke_interval=timedelta(seconds=5), timeout=timedelta(hours=1), ) + # [END howto_sensor_financial_services_operation] # [START howto_operator_financial_services_get_instance] get_instance_task = FinancialServicesGetInstanceOperator( From 6b9fe2ed98f1ff64111261847be0dffc262ca7d0 Mon Sep 17 00:00:00 2001 From: chris-law-groundtruth Date: Thu, 24 Oct 2024 22:02:30 +0100 Subject: [PATCH 15/28] add discovery doc arg to financial services operators --- .../google/cloud/hooks/financial_services.py | 11 +++------- .../cloud/operators/financial_services.py | 15 ++++++++++--- .../cloud/sensors/financial_services.py | 5 ++++- .../cloud/hooks/test_financial_services.py | 2 +- .../operators/test_financial_services.py | 9 +++++--- .../cloud/sensors/test_financial_services.py | 9 +++++--- .../example_financial_services.py | 22 ++++++++++++++++++- .../financial_services_discovery.json | 0 8 files changed, 53 insertions(+), 20 deletions(-) rename providers/{src/airflow/providers/google/cloud/hooks => tests/system/google/cloud/financial_services/resources}/financial_services_discovery.json (100%) diff --git a/providers/src/airflow/providers/google/cloud/hooks/financial_services.py b/providers/src/airflow/providers/google/cloud/hooks/financial_services.py index ed4c6b926508..aa1d5df3ce1b 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/financial_services.py +++ b/providers/src/airflow/providers/google/cloud/hooks/financial_services.py @@ -16,9 +16,6 @@ # under the License. from __future__ import annotations -import importlib.resources -import json - from googleapiclient.discovery import Resource, build_from_document from airflow.providers.google.common.hooks.base_google import GoogleBaseHook @@ -36,6 +33,7 @@ class FinancialServicesHook(GoogleBaseHook): def __init__( self, + discovery_doc: dict, gcp_conn_id: str = "google_cloud_default", **kwargs, ) -> None: @@ -49,6 +47,7 @@ def __init__( gcp_conn_id=gcp_conn_id, impersonation_chain=None, ) + self.discovery_doc = discovery_doc def get_conn(self) -> Resource: """ @@ -57,11 +56,7 @@ def get_conn(self) -> Resource: :return: a Google Cloud Financial Services service object. """ if not self.connection: - api_doc_res = importlib.resources.files("airflow.providers.google.cloud.hooks").joinpath( - "financial_services_discovery.json" - ) - with importlib.resources.as_file(api_doc_res) as file: - self.connection = build_from_document(json.loads(file.read_text())) + self.connection = build_from_document(self.discovery_doc) return self.connection diff --git a/providers/src/airflow/providers/google/cloud/operators/financial_services.py b/providers/src/airflow/providers/google/cloud/operators/financial_services.py index 1ae91df36f46..624a331a059b 100644 --- a/providers/src/airflow/providers/google/cloud/operators/financial_services.py +++ b/providers/src/airflow/providers/google/cloud/operators/financial_services.py @@ -40,7 +40,7 @@ class FinancialServicesCreateInstanceOperator(GoogleCloudBaseOperator): """ # [START howto_operator_financial_services_create_instance_template_fields] - template_fields: Sequence[str] = ("instance_id", "location_resource_uri", "kms_key_uri") + template_fields: Sequence[str] = ("instance_id", "location_resource_uri", "kms_key_uri", "discovery_doc") # [END howto_operator_financial_services_create_instance_template_fields] def __init__( @@ -48,6 +48,7 @@ def __init__( instance_id: str, location_resource_uri: str, kms_key_uri: str, + discovery_doc: dict, gcp_conn_id: str = "google_cloud_default", **kwargs, ) -> None: @@ -56,9 +57,11 @@ def __init__( self.location_resource_uri = location_resource_uri self.gcp_conn_id = gcp_conn_id self.kms_key_uri = kms_key_uri + self.discovery_doc = discovery_doc def execute(self, context: Context): hook = FinancialServicesHook( + discovery_doc=self.discovery_doc, gcp_conn_id=self.gcp_conn_id, ) response = hook.create_instance( @@ -80,21 +83,24 @@ class FinancialServicesDeleteInstanceOperator(GoogleCloudBaseOperator): """ # [START howto_operator_financial_services_get_instance_template_fields] - template_fields: Sequence[str] = ("instance_resource_uri",) + template_fields: Sequence[str] = ("instance_resource_uri", "discovery_doc") # [END howto_operator_financial_services_get_instance_template_fields] def __init__( self, instance_resource_uri: str, + discovery_doc: dict, gcp_conn_id: str = "google_cloud_default", **kwargs, ) -> None: super().__init__(**kwargs) self.instance_resource_uri = instance_resource_uri self.gcp_conn_id = gcp_conn_id + self.discovery_doc = discovery_doc def execute(self, context: Context): hook = FinancialServicesHook( + discovery_doc=self.discovery_doc, gcp_conn_id=self.gcp_conn_id, ) response = hook.delete_instance( @@ -114,21 +120,24 @@ class FinancialServicesGetInstanceOperator(GoogleCloudBaseOperator): """ # [START howto_operator_financial_services_delete_instance_template_fields] - template_fields: Sequence[str] = ("instance_resource_uri",) + template_fields: Sequence[str] = ("instance_resource_uri", "discovery_doc") # [END howto_operator_financial_services_delete_instance_template_fields] def __init__( self, instance_resource_uri: str, + discovery_doc: dict, gcp_conn_id: str = "google_cloud_default", **kwargs, ) -> None: super().__init__(**kwargs) self.instance_resource_uri = instance_resource_uri self.gcp_conn_id = gcp_conn_id + self.discovery_doc = discovery_doc def execute(self, context: Context): hook = FinancialServicesHook( + discovery_doc=self.discovery_doc, gcp_conn_id=self.gcp_conn_id, ) response = hook.get_instance( diff --git a/providers/src/airflow/providers/google/cloud/sensors/financial_services.py b/providers/src/airflow/providers/google/cloud/sensors/financial_services.py index a6dcc2f7a1c1..64ba06e7e06c 100644 --- a/providers/src/airflow/providers/google/cloud/sensors/financial_services.py +++ b/providers/src/airflow/providers/google/cloud/sensors/financial_services.py @@ -37,21 +37,24 @@ class FinancialServicesOperationSensor(BaseSensorOperator): """ # [START howto_sensor_financial_services_operation_template_fields] - template_fields: Sequence[str] = ("operation_resource_uri",) + template_fields: Sequence[str] = ("operation_resource_uri", "discovery_doc") # [END howto_sensor_financial_services_operation_template_fields] def __init__( self, operation_resource_uri: str, + discovery_doc: dict, gcp_conn_id: str = "google_cloud_default", **kwargs, ): super().__init__(**kwargs) self.operation_resource_uri = operation_resource_uri + self.discovery_doc = discovery_doc self.gcp_conn_id = gcp_conn_id def poke(self, context: Context) -> PokeReturnValue: hook = FinancialServicesHook( + discovery_doc=self.discovery_doc, gcp_conn_id=self.gcp_conn_id, ) operation = hook.get_operation(operation_resource_uri=self.operation_resource_uri) diff --git a/providers/tests/google/cloud/hooks/test_financial_services.py b/providers/tests/google/cloud/hooks/test_financial_services.py index a6290f2403cd..de618d2beeb3 100644 --- a/providers/tests/google/cloud/hooks/test_financial_services.py +++ b/providers/tests/google/cloud/hooks/test_financial_services.py @@ -46,7 +46,7 @@ def setup_method(self): "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.__init__", new=mock_init, ): - self.financial_services_hook = FinancialServicesHook() + self.financial_services_hook = FinancialServicesHook(discovery_doc={}) @patch("airflow.providers.google.cloud.hooks.financial_services.build_from_document") def test_get_conn(self, mock_build): diff --git a/providers/tests/google/cloud/operators/test_financial_services.py b/providers/tests/google/cloud/operators/test_financial_services.py index 76b29fe1ddcc..2f07a867a81b 100644 --- a/providers/tests/google/cloud/operators/test_financial_services.py +++ b/providers/tests/google/cloud/operators/test_financial_services.py @@ -42,13 +42,14 @@ def test_execute(self, mock_hook): op = FinancialServicesCreateInstanceOperator( task_id="test_create_instance_task", + discovery_doc={}, instance_id="test-instance", kms_key_uri="projects/test-project/locations/us-central1/keyRings/my-kr/cryptoKeys/my-kms-key", location_resource_uri="projects/test-project/locations/us-central1", ) op.execute(context={"ti": mock.MagicMock()}) - mock_hook.assert_called_once_with(gcp_conn_id="google_cloud_default") + mock_hook.assert_called_once_with(discovery_doc={}, gcp_conn_id="google_cloud_default") mock_hook.return_value.create_instance.assert_called_once_with( instance_id="test-instance", kms_key_uri="projects/test-project/locations/us-central1/keyRings/my-kr/cryptoKeys/my-kms-key", @@ -63,11 +64,12 @@ def test_execute(self, mock_hook): op = FinancialServicesDeleteInstanceOperator( task_id="test_delete_instance_task", + discovery_doc={}, instance_resource_uri="projects/test-project/locations/us-central1/instances/test-instance", ) op.execute(context={"ti": mock.MagicMock()}) - mock_hook.assert_called_once_with(gcp_conn_id="google_cloud_default") + mock_hook.assert_called_once_with(discovery_doc={}, gcp_conn_id="google_cloud_default") mock_hook.return_value.delete_instance.assert_called_once_with( instance_resource_uri="projects/test-project/locations/us-central1/instances/test-instance" ) @@ -80,11 +82,12 @@ def test_execute(self, mock_hook): op = FinancialServicesDeleteInstanceOperator( task_id="test_get_instance_task", + discovery_doc={}, instance_resource_uri="projects/test-project/locations/us-central1/instances/test-instance", ) op.execute(context={"ti": mock.MagicMock()}) - mock_hook.assert_called_once_with(gcp_conn_id="google_cloud_default") + mock_hook.assert_called_once_with(discovery_doc={}, gcp_conn_id="google_cloud_default") mock_hook.return_value.delete_instance.assert_called_once_with( instance_resource_uri="projects/test-project/locations/us-central1/instances/test-instance" ) diff --git a/providers/tests/google/cloud/sensors/test_financial_services.py b/providers/tests/google/cloud/sensors/test_financial_services.py index b64632e55414..0b1b8a3f3889 100644 --- a/providers/tests/google/cloud/sensors/test_financial_services.py +++ b/providers/tests/google/cloud/sensors/test_financial_services.py @@ -35,11 +35,12 @@ def test_poke(self, mock_hook): op = FinancialServicesOperationSensor( task_id="test_operation_sensor_task", + discovery_doc={}, operation_resource_uri="projects/test-project/locations/us-central1/instances/test-instance", ) response = op.poke(context={"ti": mock.MagicMock()}) - mock_hook.assert_called_once_with(gcp_conn_id="google_cloud_default") + mock_hook.assert_called_once_with(discovery_doc={}, gcp_conn_id="google_cloud_default") mock_hook.return_value.get_operation.assert_called_once_with( operation_resource_uri="projects/test-project/locations/us-central1/instances/test-instance", ) @@ -51,11 +52,12 @@ def test_poke_done(self, mock_hook): op = FinancialServicesOperationSensor( task_id="test_operation_sensor_task", + discovery_doc={}, operation_resource_uri="projects/test-project/locations/us-central1/instances/test-instance", ) response = op.poke(context={"ti": mock.MagicMock()}) - mock_hook.assert_called_once_with(gcp_conn_id="google_cloud_default") + mock_hook.assert_called_once_with(discovery_doc={}, gcp_conn_id="google_cloud_default") mock_hook.return_value.get_operation.assert_called_once_with( operation_resource_uri="projects/test-project/locations/us-central1/instances/test-instance", ) @@ -67,12 +69,13 @@ def test_poke_error(self, mock_hook): op = FinancialServicesOperationSensor( task_id="test_operation_sensor_task", + discovery_doc={}, operation_resource_uri="projects/test-project/locations/us-central1/instances/test-instance", ) with pytest.raises(AirflowFailException): op.poke(context={"ti": mock.MagicMock()}) - mock_hook.assert_called_once_with(gcp_conn_id="google_cloud_default") + mock_hook.assert_called_once_with(discovery_doc={}, gcp_conn_id="google_cloud_default") mock_hook.return_value.get_operation.assert_called_once_with( operation_resource_uri="projects/test-project/locations/us-central1/instances/test-instance", ) diff --git a/providers/tests/system/google/cloud/financial_services/example_financial_services.py b/providers/tests/system/google/cloud/financial_services/example_financial_services.py index 0db28a6f9e3b..e05595c1f239 100644 --- a/providers/tests/system/google/cloud/financial_services/example_financial_services.py +++ b/providers/tests/system/google/cloud/financial_services/example_financial_services.py @@ -16,9 +16,12 @@ # under the License. from __future__ import annotations +import json import os from datetime import datetime, timedelta +from pathlib import Path +from airflow.decorators import task from airflow.models.dag import DAG from airflow.providers.google.cloud.operators.financial_services import ( FinancialServicesCreateInstanceOperator, @@ -38,15 +41,27 @@ INSTANCE_ID = f"instance_{DAG_ID}_{ENV_ID}" +RESOURCE_DIR_PATH = str(Path(__file__).parent / "resources" / "financial_services_discovery.json") + with DAG( DAG_ID, schedule="@once", start_date=datetime(2021, 1, 1), catchup=False, ) as dag: + + @task() + def load_discovery_doc() -> str: + with open(RESOURCE_DIR_PATH) as file: + doc = json.load(file) + return doc + + discovery_doc = load_discovery_doc() + # [START howto_operator_financial_services_create_instance] create_instance_task = FinancialServicesCreateInstanceOperator( task_id="create_instance_task", + discovery_doc=discovery_doc, instance_id=INSTANCE_ID, location_resource_uri=f"projects/{PROJECT_ID}/locations/{LOCATION}", kms_key_uri=KMS_KEY, @@ -56,6 +71,7 @@ # [START howto_sensor_financial_services_operation] create_instance_sensor = FinancialServicesOperationSensor( task_id="create_instance_sensor", + discovery_doc=discovery_doc, operation_resource_uri="{{ task_instance.xcom_pull(task_ids='create_instance_task', key='return_value') }}", poke_interval=timedelta(seconds=5), timeout=timedelta(hours=1), @@ -65,6 +81,7 @@ # [START howto_operator_financial_services_get_instance] get_instance_task = FinancialServicesGetInstanceOperator( task_id="get_instance_task", + discovery_doc=discovery_doc, instance_resource_uri=f"projects/{PROJECT_ID}/locations/{LOCATION}/instances/{INSTANCE_ID}", ) # [END howto_operator_financial_services_get_instance] @@ -72,19 +89,22 @@ # [START howto_operator_financial_services_delete_instance] delete_instance_task = FinancialServicesDeleteInstanceOperator( task_id="delete_instance_task", + discovery_doc=discovery_doc, instance_resource_uri=f"projects/{PROJECT_ID}/locations/{LOCATION}/instances/{INSTANCE_ID}", ) # [END howto_operator_financial_services_delete_instance] delete_instance_sensor = FinancialServicesOperationSensor( task_id="delete_instance_sensor", + discovery_doc=discovery_doc, operation_resource_uri="{{ task_instance.xcom_pull(task_ids='delete_instance_task', key='return_value') }}", poke_interval=timedelta(seconds=5), timeout=timedelta(hours=1), ) ( - create_instance_task + discovery_doc + >> create_instance_task >> create_instance_sensor >> get_instance_task >> delete_instance_task diff --git a/providers/src/airflow/providers/google/cloud/hooks/financial_services_discovery.json b/providers/tests/system/google/cloud/financial_services/resources/financial_services_discovery.json similarity index 100% rename from providers/src/airflow/providers/google/cloud/hooks/financial_services_discovery.json rename to providers/tests/system/google/cloud/financial_services/resources/financial_services_discovery.json From a874878ace4a7a14d9152e4f0086786d6df12bd0 Mon Sep 17 00:00:00 2001 From: chris-law-groundtruth Date: Thu, 24 Oct 2024 22:08:35 +0100 Subject: [PATCH 16/28] remove example DAG --- .../example_financial_services.py | 79 ------------------- 1 file changed, 79 deletions(-) delete mode 100644 providers/src/airflow/providers/google/cloud/example_dags/example_financial_services.py diff --git a/providers/src/airflow/providers/google/cloud/example_dags/example_financial_services.py b/providers/src/airflow/providers/google/cloud/example_dags/example_financial_services.py deleted file mode 100644 index 8618ab346d84..000000000000 --- a/providers/src/airflow/providers/google/cloud/example_dags/example_financial_services.py +++ /dev/null @@ -1,79 +0,0 @@ -# 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 uuid -from datetime import datetime, timedelta - -from airflow.models.dag import DAG -from airflow.operators.python import PythonOperator -from airflow.providers.google.cloud.operators.financial_services import ( - FinancialServicesCreateInstanceOperator, - FinancialServicesGetInstanceOperator, -) -from airflow.providers.google.cloud.sensors.financial_services import FinancialServicesOperationSensor - -params = { - "project_id": None, - "location": None, - "keyRing": None, - "cryptoKey": None, -} - -with DAG( - "example_financial_services", - start_date=datetime(2024, 1, 1), - catchup=False, -) as dag: - # TODO requires an airflow variable containing the developer key / developer key secret resource URI - - generate_run_id_task = PythonOperator( - task_id="generate_run_id_task", python_callable=uuid.uuid4().__str__ - ) - - create_instance_task = FinancialServicesCreateInstanceOperator( - task_id="create_instance_task", - instance_id="instance_{{ task_instance.xcom_pull(task_ids='generate_run_id_task', key='return_value') }}", - location_resource_uri="projects/{{ params.project_id }}/locations/{{ params.location }}", - kms_key_uri="projects/{{ params.project_id }}/locations/{{ params.location }}/keyRings/{{ params.keyRing }}/cryptoKeys/{{ params.cryptoKey }}", - ) - - create_instance_sensor = FinancialServicesOperationSensor( - task_id="create_instance_sensor", - operation_resource_uri="{{ task_instance.xcom_pull(task_ids='create_instance_task', key='return_value') }}", - poke_interval=timedelta(minutes=1), - timeout=timedelta(days=1), - ) - - get_instance_task = FinancialServicesGetInstanceOperator( - task_id="get_instance_task", - instance_resource_uri="projects/{{ params.project_id }}/locations/{{ params.location }}/instances/instance_{{ task_instance.xcom_pull(task_ids='generate_run_id_task', key='return_value') }}", - ) - - # TODO Log the task info - - delete_instance_task = FinancialServicesGetInstanceOperator( - task_id="delete_instance_task", - instance_resource_uri="projects/{{ params.project_id }}/locations/{{ params.location }}/instances/instance_{{ task_instance.xcom_pull(task_ids='generate_run_id_task', key='return_value') }}", - ) - - delete_instance_sensor = FinancialServicesOperationSensor( - task_id="delete_instance_sensor", - operation_resource_uri="{{ task_instance.xcom_pull(task_ids='delete_instance_task', key='return_value') }}", - poke_interval=timedelta(minutes=1), - timeout=timedelta(days=1), - ) From 9b62db0d8c4208732b41de3b66abf3dd45c6e081 Mon Sep 17 00:00:00 2001 From: chris-law-groundtruth Date: Fri, 25 Oct 2024 10:25:25 +0100 Subject: [PATCH 17/28] fix docstrings and refactor --- .../google/cloud/hooks/financial_services.py | 20 ++++++----- .../cloud/operators/financial_services.py | 34 ++++++++++++------ .../cloud/sensors/financial_services.py | 4 ++- .../cloud/hooks/test_financial_services.py | 35 ++++++++----------- .../operators/test_financial_services.py | 27 +++++++------- .../cloud/sensors/test_financial_services.py | 14 ++++---- .../example_financial_services.py | 2 -- 7 files changed, 76 insertions(+), 60 deletions(-) diff --git a/providers/src/airflow/providers/google/cloud/hooks/financial_services.py b/providers/src/airflow/providers/google/cloud/hooks/financial_services.py index aa1d5df3ce1b..79592f8b63be 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/financial_services.py +++ b/providers/src/airflow/providers/google/cloud/hooks/financial_services.py @@ -25,6 +25,8 @@ class FinancialServicesHook(GoogleBaseHook): """ Hook for interacting with the Google Financial Services API. + :param discovery_doc: Discovery document for building the Financial Services API + as described `here `__ :param gcp_conn_id: Identifier of connection to Google Cloud Platform. Defaults to "google_cloud_default". """ @@ -53,7 +55,7 @@ def get_conn(self) -> Resource: """ Establish a connection to the Google Financial Services API. - :return: a Google Cloud Financial Services service object. + :return: A Google Cloud Financial Services API service resource. """ if not self.connection: self.connection = build_from_document(self.discovery_doc) @@ -62,10 +64,10 @@ def get_conn(self) -> Resource: def get_instance(self, instance_resource_uri: str) -> dict: """ - Get an AML AI instance. + Get a Financial Services Anti-Money Laundering AI instance. :param instance_resource_uri: URI of the instance to get (format: - 'projects//locations//instances/) + 'projects//locations//instances/) :returns: A dictionary containing the instance metadata """ @@ -75,7 +77,7 @@ def get_instance(self, instance_resource_uri: str) -> dict: def create_instance(self, instance_id: str, kms_key_uri: str, location_resource_uri: str) -> dict: """ - Create an AML AI instance. + Create a Financial Services Anti-Money Laundering AI instance. :param instance_id: Identifier for the instance to create :param kms_key_uri: URI of the KMS key to that will be used for instance encryption @@ -100,10 +102,10 @@ def create_instance(self, instance_id: str, kms_key_uri: str, location_resource_ def delete_instance(self, instance_resource_uri: str) -> dict: """ - Delete an AML AI instance. + Delete a Financial Services Anti-Money Laundering AI instance. :param instance_resource_uri: URI of the instance to delete (format: - 'projects//locations//instances/) + 'projects//locations//instances/) :returns: A dictionary containing metadata for the delete instance operation @@ -114,9 +116,11 @@ def delete_instance(self, instance_resource_uri: str) -> dict: def get_operation(self, operation_resource_uri: str) -> dict: """ - Get an operation. + Get a Financial Services Anti-Money Laundering AI operation. + + :param operation_resource_uri: URI of the operation to get (format: + 'projects//locations//operations/) - :param operation_resource_uri: URI of the operation :return: A dictionary containing metadata for the operation """ conn = self.get_conn() diff --git a/providers/src/airflow/providers/google/cloud/operators/financial_services.py b/providers/src/airflow/providers/google/cloud/operators/financial_services.py index 624a331a059b..e5c449e67b51 100644 --- a/providers/src/airflow/providers/google/cloud/operators/financial_services.py +++ b/providers/src/airflow/providers/google/cloud/operators/financial_services.py @@ -27,7 +27,7 @@ class FinancialServicesCreateInstanceOperator(GoogleCloudBaseOperator): """ - Create a Financial Services AML AI Instance. + Create a Financial Services Anti-Money Laundering AI instance. :param instance_id: Identifier for the instance to create :param location_resource_uri: URI of the location to create the instance in @@ -35,12 +35,20 @@ class FinancialServicesCreateInstanceOperator(GoogleCloudBaseOperator): :param kms_key_uri: URI of the KMS key to that will be used for instance encryption (format: 'projects//locations//keyRings/ /cryptoKeys/') + :param discovery_doc: Discovery document for building the Financial Services API + as described `here `__ :param gcp_conn_id: Identifier of connection to Google Cloud Platform. Defaults to "google_cloud_default". """ # [START howto_operator_financial_services_create_instance_template_fields] - template_fields: Sequence[str] = ("instance_id", "location_resource_uri", "kms_key_uri", "discovery_doc") + template_fields: Sequence[str] = ( + "instance_id", + "location_resource_uri", + "kms_key_uri", + "discovery_doc", + "gcp_conn_id", + ) # [END howto_operator_financial_services_create_instance_template_fields] def __init__( @@ -55,9 +63,9 @@ def __init__( super().__init__(**kwargs) self.instance_id = instance_id self.location_resource_uri = location_resource_uri - self.gcp_conn_id = gcp_conn_id self.kms_key_uri = kms_key_uri self.discovery_doc = discovery_doc + self.gcp_conn_id = gcp_conn_id def execute(self, context: Context): hook = FinancialServicesHook( @@ -74,16 +82,18 @@ def execute(self, context: Context): class FinancialServicesDeleteInstanceOperator(GoogleCloudBaseOperator): """ - Delete a Financial Services AML AI Instance. + Delete a Financial Services Anti-Money Laundering AI instance. :param instance_resource_uri: URI of the instance to delete (format: - 'projects//locations//instances/) + 'projects//locations//instances/) + :param discovery_doc: Discovery document for building the Financial Services API + as described `here `__ :param gcp_conn_id: Identifier of connection to Google Cloud Platform. Defaults to "google_cloud_default". """ # [START howto_operator_financial_services_get_instance_template_fields] - template_fields: Sequence[str] = ("instance_resource_uri", "discovery_doc") + template_fields: Sequence[str] = ("instance_resource_uri", "discovery_doc", "gcp_conn_id") # [END howto_operator_financial_services_get_instance_template_fields] def __init__( @@ -95,8 +105,8 @@ def __init__( ) -> None: super().__init__(**kwargs) self.instance_resource_uri = instance_resource_uri - self.gcp_conn_id = gcp_conn_id self.discovery_doc = discovery_doc + self.gcp_conn_id = gcp_conn_id def execute(self, context: Context): hook = FinancialServicesHook( @@ -111,16 +121,18 @@ def execute(self, context: Context): class FinancialServicesGetInstanceOperator(GoogleCloudBaseOperator): """ - Get a Financial Services AML AI Instance. + Get a Financial Services Anti-Money Laundering AI instance. :param instance_resource_uri: URI of the instance to get (format: - 'projects//locations//instances/) + 'projects//locations//instances/) + :param discovery_doc: Discovery document for building the Financial Services API + as described `here `__ :param gcp_conn_id: Identifier of connection to Google Cloud Platform. Defaults to "google_cloud_default". """ # [START howto_operator_financial_services_delete_instance_template_fields] - template_fields: Sequence[str] = ("instance_resource_uri", "discovery_doc") + template_fields: Sequence[str] = ("instance_resource_uri", "discovery_doc", "gcp_conn_id") # [END howto_operator_financial_services_delete_instance_template_fields] def __init__( @@ -132,8 +144,8 @@ def __init__( ) -> None: super().__init__(**kwargs) self.instance_resource_uri = instance_resource_uri - self.gcp_conn_id = gcp_conn_id self.discovery_doc = discovery_doc + self.gcp_conn_id = gcp_conn_id def execute(self, context: Context): hook = FinancialServicesHook( diff --git a/providers/src/airflow/providers/google/cloud/sensors/financial_services.py b/providers/src/airflow/providers/google/cloud/sensors/financial_services.py index 64ba06e7e06c..d7970c76fcbc 100644 --- a/providers/src/airflow/providers/google/cloud/sensors/financial_services.py +++ b/providers/src/airflow/providers/google/cloud/sensors/financial_services.py @@ -32,12 +32,14 @@ class FinancialServicesOperationSensor(BaseSensorOperator): :param operation_resource_uri: URI of the operation (format: 'projects//locations//operations/) + :param discovery_doc: Discovery document for building the Financial Services API + as described `here `__ :param gcp_conn_id: Identifier of connection to Google Cloud Platform. Defaults to "google_cloud_default". """ # [START howto_sensor_financial_services_operation_template_fields] - template_fields: Sequence[str] = ("operation_resource_uri", "discovery_doc") + template_fields: Sequence[str] = ("operation_resource_uri", "discovery_doc", "gcp_conn_id") # [END howto_sensor_financial_services_operation_template_fields] def __init__( diff --git a/providers/tests/google/cloud/hooks/test_financial_services.py b/providers/tests/google/cloud/hooks/test_financial_services.py index de618d2beeb3..f8a577e16b91 100644 --- a/providers/tests/google/cloud/hooks/test_financial_services.py +++ b/providers/tests/google/cloud/hooks/test_financial_services.py @@ -20,15 +20,18 @@ from airflow.providers.google.cloud.hooks.financial_services import FinancialServicesHook +TEST_KMS_KEY_URI = "projects/test-project/locations/us-central1/keyRings/my-kr/cryptoKeys/my-kms-key" +TEST_LOCATION_RESOURCE_URI = "projects/test-project/locations/us-central1" +TEST_INSTANCE_ID = "test-instance" +TEST_INSTANCE_RESOURCE_URI = f"{TEST_LOCATION_RESOURCE_URI}/instances/{TEST_INSTANCE_ID}" TEST_OPERATION = {"name": "test-operation", "metadata": {}, "done": False} - TEST_INSTANCE = { "name": "test-instance", "createTime": "2014-10-02T15:01:23Z", "updateTime": "2014-10-02T15:01:23Z", "labels": {}, "state": "ACTIVE", - "kmsKey": "projects/test-project/locations/us-central1/keyRings/my-kr/cryptoKeys/my-kms-key", + "kmsKey": TEST_KMS_KEY_URI, } @@ -65,13 +68,9 @@ def test_get_instance(self, mock_get_conn): instances = locations.instances.return_value instances.get.return_value.execute.return_value = TEST_INSTANCE - response = self.financial_services_hook.get_instance( - instance_resource_uri="projects/test-project/locations/us-central1/instances/unit-test-instance" - ) + response = self.financial_services_hook.get_instance(instance_resource_uri=TEST_INSTANCE_RESOURCE_URI) - instances.get.assert_called_once_with( - name="projects/test-project/locations/us-central1/instances/unit-test-instance" - ) + instances.get.assert_called_once_with(name=TEST_INSTANCE_RESOURCE_URI) assert response == TEST_INSTANCE @@ -85,17 +84,15 @@ def test_create_instance(self, mock_get_conn): instances.create.return_value.execute.return_value = TEST_OPERATION response = self.financial_services_hook.create_instance( - instance_id="test-instance", - kms_key_uri="projects/test-project/locations/us-central1/keyRings/my-kr/cryptoKeys/my-kms-key", - location_resource_uri="projects/test-project/locations/us-central1", + instance_id=TEST_INSTANCE_ID, + kms_key_uri=TEST_KMS_KEY_URI, + location_resource_uri=TEST_LOCATION_RESOURCE_URI, ) instances.create.assert_called_once_with( - parent="projects/test-project/locations/us-central1", - instanceId="test-instance", - body={ - "kmsKey": "projects/test-project/locations/us-central1/keyRings/my-kr/cryptoKeys/my-kms-key" - }, + parent=TEST_LOCATION_RESOURCE_URI, + instanceId=TEST_INSTANCE_ID, + body={"kmsKey": TEST_KMS_KEY_URI}, ) assert response == TEST_OPERATION @@ -110,11 +107,9 @@ def test_delete_instance(self, mock_get_conn): instances.delete.return_value.execute.return_value = TEST_OPERATION response = self.financial_services_hook.delete_instance( - instance_resource_uri="projects/test-project/locations/us-central1/instances/unit-test-instance" + instance_resource_uri=TEST_INSTANCE_RESOURCE_URI ) - instances.delete.assert_called_once_with( - name="projects/test-project/locations/us-central1/instances/unit-test-instance" - ) + instances.delete.assert_called_once_with(name=TEST_INSTANCE_RESOURCE_URI) assert response == TEST_OPERATION diff --git a/providers/tests/google/cloud/operators/test_financial_services.py b/providers/tests/google/cloud/operators/test_financial_services.py index 2f07a867a81b..2d54d91fbb4a 100644 --- a/providers/tests/google/cloud/operators/test_financial_services.py +++ b/providers/tests/google/cloud/operators/test_financial_services.py @@ -23,15 +23,18 @@ FinancialServicesDeleteInstanceOperator, ) +TEST_KMS_KEY_URI = "projects/test-project/locations/us-central1/keyRings/my-kr/cryptoKeys/my-kms-key" +TEST_LOCATION_RESOURCE_URI = "projects/test-project/locations/us-central1" +TEST_INSTANCE_ID = "test-instance" +TEST_INSTANCE_RESOURCE_URI = f"{TEST_LOCATION_RESOURCE_URI}/instances/{TEST_INSTANCE_ID}" TEST_OPERATION = {"name": "test-operation", "metadata": {}, "done": False} - TEST_INSTANCE = { "name": "test-instance", "createTime": "2014-10-02T15:01:23Z", "updateTime": "2014-10-02T15:01:23Z", "labels": {}, "state": "ACTIVE", - "kmsKey": "projects/test-project/locations/us-central1/keyRings/my-kr/cryptoKeys/my-kms-key", + "kmsKey": TEST_KMS_KEY_URI, } @@ -43,17 +46,17 @@ def test_execute(self, mock_hook): op = FinancialServicesCreateInstanceOperator( task_id="test_create_instance_task", discovery_doc={}, - instance_id="test-instance", - kms_key_uri="projects/test-project/locations/us-central1/keyRings/my-kr/cryptoKeys/my-kms-key", - location_resource_uri="projects/test-project/locations/us-central1", + instance_id=TEST_INSTANCE_ID, + kms_key_uri=TEST_KMS_KEY_URI, + location_resource_uri=TEST_LOCATION_RESOURCE_URI, ) op.execute(context={"ti": mock.MagicMock()}) mock_hook.assert_called_once_with(discovery_doc={}, gcp_conn_id="google_cloud_default") mock_hook.return_value.create_instance.assert_called_once_with( - instance_id="test-instance", - kms_key_uri="projects/test-project/locations/us-central1/keyRings/my-kr/cryptoKeys/my-kms-key", - location_resource_uri="projects/test-project/locations/us-central1", + instance_id=TEST_INSTANCE_ID, + kms_key_uri=TEST_KMS_KEY_URI, + location_resource_uri=TEST_LOCATION_RESOURCE_URI, ) @@ -65,13 +68,13 @@ def test_execute(self, mock_hook): op = FinancialServicesDeleteInstanceOperator( task_id="test_delete_instance_task", discovery_doc={}, - instance_resource_uri="projects/test-project/locations/us-central1/instances/test-instance", + instance_resource_uri=TEST_INSTANCE_RESOURCE_URI, ) op.execute(context={"ti": mock.MagicMock()}) mock_hook.assert_called_once_with(discovery_doc={}, gcp_conn_id="google_cloud_default") mock_hook.return_value.delete_instance.assert_called_once_with( - instance_resource_uri="projects/test-project/locations/us-central1/instances/test-instance" + instance_resource_uri=TEST_INSTANCE_RESOURCE_URI ) @@ -83,11 +86,11 @@ def test_execute(self, mock_hook): op = FinancialServicesDeleteInstanceOperator( task_id="test_get_instance_task", discovery_doc={}, - instance_resource_uri="projects/test-project/locations/us-central1/instances/test-instance", + instance_resource_uri=TEST_INSTANCE_RESOURCE_URI, ) op.execute(context={"ti": mock.MagicMock()}) mock_hook.assert_called_once_with(discovery_doc={}, gcp_conn_id="google_cloud_default") mock_hook.return_value.delete_instance.assert_called_once_with( - instance_resource_uri="projects/test-project/locations/us-central1/instances/test-instance" + instance_resource_uri=TEST_INSTANCE_RESOURCE_URI ) diff --git a/providers/tests/google/cloud/sensors/test_financial_services.py b/providers/tests/google/cloud/sensors/test_financial_services.py index 0b1b8a3f3889..d115f548a11a 100644 --- a/providers/tests/google/cloud/sensors/test_financial_services.py +++ b/providers/tests/google/cloud/sensors/test_financial_services.py @@ -23,6 +23,8 @@ from airflow.exceptions import AirflowFailException from airflow.providers.google.cloud.sensors.financial_services import FinancialServicesOperationSensor +TEST_LOCATION_RESOURCE_URI = "projects/test-project/locations/us-central1" +TEST_OPERATION_RESOURCE_URI = f"{TEST_LOCATION_RESOURCE_URI}/operations/test-operation" TEST_OPERATION = {"name": "test-operation", "metadata": {}, "done": False} TEST_OPERATION_DONE = {"name": "test-operation", "metadata": {}, "done": True, "response": {}} TEST_OPERATION_ERROR = {"name": "test-operation", "metadata": {}, "done": True, "error": {}} @@ -36,13 +38,13 @@ def test_poke(self, mock_hook): op = FinancialServicesOperationSensor( task_id="test_operation_sensor_task", discovery_doc={}, - operation_resource_uri="projects/test-project/locations/us-central1/instances/test-instance", + operation_resource_uri=TEST_OPERATION_RESOURCE_URI, ) response = op.poke(context={"ti": mock.MagicMock()}) mock_hook.assert_called_once_with(discovery_doc={}, gcp_conn_id="google_cloud_default") mock_hook.return_value.get_operation.assert_called_once_with( - operation_resource_uri="projects/test-project/locations/us-central1/instances/test-instance", + operation_resource_uri=TEST_OPERATION_RESOURCE_URI, ) assert response.is_done == TEST_OPERATION["done"] @@ -53,13 +55,13 @@ def test_poke_done(self, mock_hook): op = FinancialServicesOperationSensor( task_id="test_operation_sensor_task", discovery_doc={}, - operation_resource_uri="projects/test-project/locations/us-central1/instances/test-instance", + operation_resource_uri=TEST_OPERATION_RESOURCE_URI, ) response = op.poke(context={"ti": mock.MagicMock()}) mock_hook.assert_called_once_with(discovery_doc={}, gcp_conn_id="google_cloud_default") mock_hook.return_value.get_operation.assert_called_once_with( - operation_resource_uri="projects/test-project/locations/us-central1/instances/test-instance", + operation_resource_uri=TEST_OPERATION_RESOURCE_URI, ) assert response.is_done == TEST_OPERATION_DONE["done"] @@ -70,12 +72,12 @@ def test_poke_error(self, mock_hook): op = FinancialServicesOperationSensor( task_id="test_operation_sensor_task", discovery_doc={}, - operation_resource_uri="projects/test-project/locations/us-central1/instances/test-instance", + operation_resource_uri=TEST_OPERATION_RESOURCE_URI, ) with pytest.raises(AirflowFailException): op.poke(context={"ti": mock.MagicMock()}) mock_hook.assert_called_once_with(discovery_doc={}, gcp_conn_id="google_cloud_default") mock_hook.return_value.get_operation.assert_called_once_with( - operation_resource_uri="projects/test-project/locations/us-central1/instances/test-instance", + operation_resource_uri=TEST_OPERATION_RESOURCE_URI, ) diff --git a/providers/tests/system/google/cloud/financial_services/example_financial_services.py b/providers/tests/system/google/cloud/financial_services/example_financial_services.py index e05595c1f239..fb15a3caeab8 100644 --- a/providers/tests/system/google/cloud/financial_services/example_financial_services.py +++ b/providers/tests/system/google/cloud/financial_services/example_financial_services.py @@ -38,9 +38,7 @@ KMS_KEY = os.environ.get("SYSTEM_TESTS_GCP_KMS_KEY") DAG_ID = "financial_services_instance" - INSTANCE_ID = f"instance_{DAG_ID}_{ENV_ID}" - RESOURCE_DIR_PATH = str(Path(__file__).parent / "resources" / "financial_services_discovery.json") with DAG( From 188de9092694edd83c91e500688bb32867fc4993 Mon Sep 17 00:00:00 2001 From: claw89 Date: Wed, 30 Oct 2024 10:24:49 +0000 Subject: [PATCH 18/28] fix nullable KMS_KEY arg --- .../cloud/financial_services/example_financial_services.py | 1 + 1 file changed, 1 insertion(+) diff --git a/providers/tests/system/google/cloud/financial_services/example_financial_services.py b/providers/tests/system/google/cloud/financial_services/example_financial_services.py index fb15a3caeab8..fbdd928c5002 100644 --- a/providers/tests/system/google/cloud/financial_services/example_financial_services.py +++ b/providers/tests/system/google/cloud/financial_services/example_financial_services.py @@ -36,6 +36,7 @@ PROJECT_ID = os.environ.get("SYSTEM_TESTS_GCP_PROJECT") or DEFAULT_GCP_SYSTEM_TEST_PROJECT_ID LOCATION = os.environ.get("SYSTEM_TESTS_GCP_LOCATION", "us-central1") KMS_KEY = os.environ.get("SYSTEM_TESTS_GCP_KMS_KEY") +assert KMS_KEY is not None DAG_ID = "financial_services_instance" INSTANCE_ID = f"instance_{DAG_ID}_{ENV_ID}" From e8f5339123e14cfc3808a70c128c91bcf39683cb Mon Sep 17 00:00:00 2001 From: claw89 Date: Wed, 30 Oct 2024 10:47:43 +0000 Subject: [PATCH 19/28] fix documentation --- .../operators/cloud/financial_services.rst | 7 +------ .../google/cloud/operators/financial_services.py | 12 ++++++++++++ .../google/cloud/sensors/financial_services.py | 4 ++++ 3 files changed, 17 insertions(+), 6 deletions(-) diff --git a/docs/apache-airflow-providers-google/operators/cloud/financial_services.rst b/docs/apache-airflow-providers-google/operators/cloud/financial_services.rst index e51ec04a9d02..367e5f3871de 100644 --- a/docs/apache-airflow-providers-google/operators/cloud/financial_services.rst +++ b/docs/apache-airflow-providers-google/operators/cloud/financial_services.rst @@ -60,7 +60,6 @@ More information See Google Cloud Financial Services API documentation to `create an instance `_. -.. _howto/operator:FinancialServicesCreateInstanceOperator: .. _howto/operator:FinancialServicesGetInstanceOperator: @@ -95,7 +94,6 @@ More information See Google Cloud Financial Services API documentation to `get an instance `_. -.. _howto/operator:FinancialServicesGetInstanceOperator: .. _howto/operator:FinancialServicesDeleteInstanceOperator: @@ -130,7 +128,6 @@ More information See Google Cloud Financial Services API documentation to `delete an instance `_. -.. _howto/operator:FinancialServicesDeleteInstanceOperator: Sensors @@ -156,7 +153,7 @@ Using the sensor Templating """""""""" -.. literalinclude:: /../../providers/src/airflow/providers/google/cloud/operators/financial_services.py +.. literalinclude:: /../../providers/src/airflow/providers/google/cloud/sensors/financial_services.py :language: python :dedent: 4 :start-after: [START howto_sensor_financial_services_operation_template_fields] @@ -167,5 +164,3 @@ More information See Google Cloud Financial Services API documentation to `get an operation `_. - -.. _howto/sensor:FinancialServicesOperationSensor: diff --git a/providers/src/airflow/providers/google/cloud/operators/financial_services.py b/providers/src/airflow/providers/google/cloud/operators/financial_services.py index e5c449e67b51..4ec0dcff8bf5 100644 --- a/providers/src/airflow/providers/google/cloud/operators/financial_services.py +++ b/providers/src/airflow/providers/google/cloud/operators/financial_services.py @@ -29,6 +29,10 @@ class FinancialServicesCreateInstanceOperator(GoogleCloudBaseOperator): """ Create a Financial Services Anti-Money Laundering AI instance. + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:FinancialServicesCreateInstanceOperator` + :param instance_id: Identifier for the instance to create :param location_resource_uri: URI of the location to create the instance in (format: 'projects//locations/) @@ -84,6 +88,10 @@ class FinancialServicesDeleteInstanceOperator(GoogleCloudBaseOperator): """ Delete a Financial Services Anti-Money Laundering AI instance. + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:FinancialServicesDeleteInstanceOperator` + :param instance_resource_uri: URI of the instance to delete (format: 'projects//locations//instances/) :param discovery_doc: Discovery document for building the Financial Services API @@ -123,6 +131,10 @@ class FinancialServicesGetInstanceOperator(GoogleCloudBaseOperator): """ Get a Financial Services Anti-Money Laundering AI instance. + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:FinancialServicesGetInstanceOperator` + :param instance_resource_uri: URI of the instance to get (format: 'projects//locations//instances/) :param discovery_doc: Discovery document for building the Financial Services API diff --git a/providers/src/airflow/providers/google/cloud/sensors/financial_services.py b/providers/src/airflow/providers/google/cloud/sensors/financial_services.py index d7970c76fcbc..13d405e03637 100644 --- a/providers/src/airflow/providers/google/cloud/sensors/financial_services.py +++ b/providers/src/airflow/providers/google/cloud/sensors/financial_services.py @@ -30,6 +30,10 @@ class FinancialServicesOperationSensor(BaseSensorOperator): """ Check status of AML AI operation. + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/sensor:FinancialServicesOperationSensor` + :param operation_resource_uri: URI of the operation (format: 'projects//locations//operations/) :param discovery_doc: Discovery document for building the Financial Services API From 89fa2e72ec4b409277d29eb418ded715240c4bfa Mon Sep 17 00:00:00 2001 From: claw89 Date: Wed, 30 Oct 2024 16:17:30 +0000 Subject: [PATCH 20/28] remove delegate_to deprecation check --- .../providers/google/cloud/hooks/financial_services.py | 6 ------ 1 file changed, 6 deletions(-) diff --git a/providers/src/airflow/providers/google/cloud/hooks/financial_services.py b/providers/src/airflow/providers/google/cloud/hooks/financial_services.py index 79592f8b63be..7ff837d01918 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/financial_services.py +++ b/providers/src/airflow/providers/google/cloud/hooks/financial_services.py @@ -39,12 +39,6 @@ def __init__( gcp_conn_id: str = "google_cloud_default", **kwargs, ) -> None: - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally" - " removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) super().__init__( gcp_conn_id=gcp_conn_id, impersonation_chain=None, From 48ee457dc1fab85a67b4de743fdeb7f947fc8db1 Mon Sep 17 00:00:00 2001 From: claw89 Date: Wed, 30 Oct 2024 16:32:04 +0000 Subject: [PATCH 21/28] divide resource names into parts in args --- .../google/cloud/hooks/financial_services.py | 56 ++++++++++++------- 1 file changed, 37 insertions(+), 19 deletions(-) diff --git a/providers/src/airflow/providers/google/cloud/hooks/financial_services.py b/providers/src/airflow/providers/google/cloud/hooks/financial_services.py index 7ff837d01918..c64002adae91 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/financial_services.py +++ b/providers/src/airflow/providers/google/cloud/hooks/financial_services.py @@ -56,67 +56,85 @@ def get_conn(self) -> Resource: return self.connection - def get_instance(self, instance_resource_uri: str) -> dict: + def get_instance(self, project_id: str, region: str, instance_id: str) -> dict: """ Get a Financial Services Anti-Money Laundering AI instance. - :param instance_resource_uri: URI of the instance to get (format: - 'projects//locations//instances/) + :param project_id: Required. The ID of the Google Cloud project that the service belongs to. + :param region: Required. The ID of the Google Cloud region that the service belongs to. + :param instance_id: Required. The ID of the instance, which is used as the final component of the + instances's name. :returns: A dictionary containing the instance metadata """ conn = self.get_conn() - response = conn.projects().locations().instances().get(name=instance_resource_uri).execute() + name = f"projects/{project_id}/locations/{region}/instances/{instance_id}" + response = conn.projects().locations().instances().get(name=name).execute() return response - def create_instance(self, instance_id: str, kms_key_uri: str, location_resource_uri: str) -> dict: + def create_instance( + self, project_id: str, region: str, instance_id: str, kms_key_ring_id: str, kms_key_id: str + ) -> dict: """ Create a Financial Services Anti-Money Laundering AI instance. - :param instance_id: Identifier for the instance to create - :param kms_key_uri: URI of the KMS key to that will be used for instance encryption - (format: 'projects//locations//keyRings// - cryptoKeys/') + :param project_id: Required. The ID of the Google Cloud project that the service belongs to. + :param region: Required. The ID of the Google Cloud region that the service belongs to. + :param instance_id: Required. The ID of the instance, which is used as the final component of the + instances's name. + :param kms_key_ring_id: Required. The ID of the Google Cloud KMS key ring containing the key to + use for instance encryption + :param kms_key_id: Required. The ID of the Google Cloud KMS key to use for instance encryption :returns: A dictionary containing metadata for the create instance operation """ conn = self.get_conn() + parent = f"projects/{project_id}/locations/{region}" + kms_key = ( + f"projects/{project_id}/locations/{region}/keyRings/{kms_key_ring_id}/cryptoKeys{kms_key_id}" + ) response = ( conn.projects() .locations() .instances() .create( - parent=location_resource_uri, + parent=parent, instanceId=instance_id, - body={"kmsKey": kms_key_uri}, + body={"kmsKey": kms_key}, ) .execute() ) return response - def delete_instance(self, instance_resource_uri: str) -> dict: + def delete_instance(self, project_id: str, region: str, instance_id: str) -> dict: """ Delete a Financial Services Anti-Money Laundering AI instance. - :param instance_resource_uri: URI of the instance to delete (format: - 'projects//locations//instances/) + :param project_id: Required. The ID of the Google Cloud project that the service belongs to. + :param region: Required. The ID of the Google Cloud region that the service belongs to. + :param instance_id: Required. The ID of the instance, which is used as the final component of the + instances's name. :returns: A dictionary containing metadata for the delete instance operation """ conn = self.get_conn() - response = conn.projects().locations().instances().delete(name=instance_resource_uri).execute() + name = f"projects/{project_id}/locations/{region}/instances/{instance_id}" + response = conn.projects().locations().instances().delete(name=name).execute() return response - def get_operation(self, operation_resource_uri: str) -> dict: + def get_operation(self, project_id: str, region: str, operation_id: str) -> dict: """ Get a Financial Services Anti-Money Laundering AI operation. - :param operation_resource_uri: URI of the operation to get (format: - 'projects//locations//operations/) + :param project_id: Required. The ID of the Google Cloud project that the service belongs to. + :param region: Required. The ID of the Google Cloud region that the service belongs to. + :param operation_id: Required. The ID of the operation, which is used as the final component of the + operation's name. :return: A dictionary containing metadata for the operation """ conn = self.get_conn() - response = conn.projects().locations().operations().get(name=operation_resource_uri).execute() + name = f"projects/{project_id}/locations/{region}/operations/{operation_id}" + response = conn.projects().locations().operations().get(name=name).execute() return response From 628b1a59e10fe301ab3ac74dc372799df85f4ccc Mon Sep 17 00:00:00 2001 From: claw89 Date: Wed, 30 Oct 2024 16:49:33 +0000 Subject: [PATCH 22/28] divide resource names in operators --- .../cloud/operators/financial_services.py | 69 +++++++++++++------ 1 file changed, 47 insertions(+), 22 deletions(-) diff --git a/providers/src/airflow/providers/google/cloud/operators/financial_services.py b/providers/src/airflow/providers/google/cloud/operators/financial_services.py index 4ec0dcff8bf5..40f9220d4098 100644 --- a/providers/src/airflow/providers/google/cloud/operators/financial_services.py +++ b/providers/src/airflow/providers/google/cloud/operators/financial_services.py @@ -33,12 +33,13 @@ class FinancialServicesCreateInstanceOperator(GoogleCloudBaseOperator): For more information on how to use this operator, take a look at the guide: :ref:`howto/operator:FinancialServicesCreateInstanceOperator` - :param instance_id: Identifier for the instance to create - :param location_resource_uri: URI of the location to create the instance in - (format: 'projects//locations/) - :param kms_key_uri: URI of the KMS key to that will be used for instance - encryption (format: 'projects//locations//keyRings/ - /cryptoKeys/') + :param project_id: Required. The ID of the Google Cloud project that the service belongs to. + :param region: Required. The ID of the Google Cloud region that the service belongs to. + :param instance_id: Required. The ID of the instance, which is used as the final component of the + instances's name. + :param kms_key_ring_id: Required. The ID of the Google Cloud KMS key ring containing the key to + use for instance encryption + :param kms_key_id: Required. The ID of the Google Cloud KMS key to use for instance encryption :param discovery_doc: Discovery document for building the Financial Services API as described `here `__ :param gcp_conn_id: Identifier of connection to Google Cloud Platform. @@ -47,9 +48,11 @@ class FinancialServicesCreateInstanceOperator(GoogleCloudBaseOperator): # [START howto_operator_financial_services_create_instance_template_fields] template_fields: Sequence[str] = ( + "project_id", + "region", "instance_id", - "location_resource_uri", - "kms_key_uri", + "kms_key_ring_id", + "kms_key_id", "discovery_doc", "gcp_conn_id", ) @@ -57,17 +60,21 @@ class FinancialServicesCreateInstanceOperator(GoogleCloudBaseOperator): def __init__( self, + project_id: str, + region: str, instance_id: str, - location_resource_uri: str, - kms_key_uri: str, + kms_key_ring_id: str, + kms_key_id: str, discovery_doc: dict, gcp_conn_id: str = "google_cloud_default", **kwargs, ) -> None: super().__init__(**kwargs) + self.project_id = project_id + self.region = region self.instance_id = instance_id - self.location_resource_uri = location_resource_uri - self.kms_key_uri = kms_key_uri + self.kms_key_ring_id = kms_key_ring_id + self.kms_key_id = kms_key_id self.discovery_doc = discovery_doc self.gcp_conn_id = gcp_conn_id @@ -76,10 +83,14 @@ def execute(self, context: Context): discovery_doc=self.discovery_doc, gcp_conn_id=self.gcp_conn_id, ) + self.log.info("Creating Financial Services instance: %s", self.instance_id) + response = hook.create_instance( + project_id=self.project_id, + region=self.region, instance_id=self.instance_id, - kms_key_uri=self.kms_key_uri, - location_resource_uri=self.location_resource_uri, + kms_key_ring_id=self.kms_key_ring_id, + kms_key_id=self.kms_key_id, ) return response["name"] @@ -101,18 +112,22 @@ class FinancialServicesDeleteInstanceOperator(GoogleCloudBaseOperator): """ # [START howto_operator_financial_services_get_instance_template_fields] - template_fields: Sequence[str] = ("instance_resource_uri", "discovery_doc", "gcp_conn_id") + template_fields: Sequence[str] = ("project_id", "region", "instance_id", "discovery_doc", "gcp_conn_id") # [END howto_operator_financial_services_get_instance_template_fields] def __init__( self, - instance_resource_uri: str, + project_id: str, + region: str, + instance_id: str, discovery_doc: dict, gcp_conn_id: str = "google_cloud_default", **kwargs, ) -> None: super().__init__(**kwargs) - self.instance_resource_uri = instance_resource_uri + self.project_id = project_id + self.region = region + self.instance_id = instance_id self.discovery_doc = discovery_doc self.gcp_conn_id = gcp_conn_id @@ -121,8 +136,12 @@ def execute(self, context: Context): discovery_doc=self.discovery_doc, gcp_conn_id=self.gcp_conn_id, ) + self.log.info("Deleting Financial Services instance: %s", self.instance_id) + response = hook.delete_instance( - instance_resource_uri=self.instance_resource_uri, + project_id=self.project_id, + region=self.region, + instance_id=self.instance_id, ) return response["name"] @@ -144,18 +163,22 @@ class FinancialServicesGetInstanceOperator(GoogleCloudBaseOperator): """ # [START howto_operator_financial_services_delete_instance_template_fields] - template_fields: Sequence[str] = ("instance_resource_uri", "discovery_doc", "gcp_conn_id") + template_fields: Sequence[str] = ("project_id", "region", "instance_id", "discovery_doc", "gcp_conn_id") # [END howto_operator_financial_services_delete_instance_template_fields] def __init__( self, - instance_resource_uri: str, + project_id: str, + region: str, + instance_id: str, discovery_doc: dict, gcp_conn_id: str = "google_cloud_default", **kwargs, ) -> None: super().__init__(**kwargs) - self.instance_resource_uri = instance_resource_uri + self.project_id = project_id + self.region = region + self.instance_id = instance_id self.discovery_doc = discovery_doc self.gcp_conn_id = gcp_conn_id @@ -164,7 +187,9 @@ def execute(self, context: Context): discovery_doc=self.discovery_doc, gcp_conn_id=self.gcp_conn_id, ) + self.log.info("Fetching Financial Services instance: %s", self.instance_id) + response = hook.get_instance( - instance_resource_uri=self.instance_resource_uri, + project_id=self.project_id, region=self.region, instance_id=self.instance_id ) return response From 13953156efe8e9223b53d15c6e8dff787b56e2bf Mon Sep 17 00:00:00 2001 From: claw89 Date: Thu, 31 Oct 2024 15:45:28 +0000 Subject: [PATCH 23/28] use google api_core Operation type --- .../google/cloud/hooks/financial_services.py | 56 +++++++++++++++++-- .../cloud/operators/financial_services.py | 35 ++++++++---- 2 files changed, 75 insertions(+), 16 deletions(-) diff --git a/providers/src/airflow/providers/google/cloud/hooks/financial_services.py b/providers/src/airflow/providers/google/cloud/hooks/financial_services.py index c64002adae91..188508407479 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/financial_services.py +++ b/providers/src/airflow/providers/google/cloud/hooks/financial_services.py @@ -16,8 +16,13 @@ # under the License. from __future__ import annotations +from google.api_core import operations_v1 +from google.api_core.future import polling +from google.api_core.operation import Operation +from google.protobuf.empty_pb2 import Empty from googleapiclient.discovery import Resource, build_from_document +from airflow.exceptions import AirflowException from airflow.providers.google.common.hooks.base_google import GoogleBaseHook @@ -45,6 +50,18 @@ def __init__( ) self.discovery_doc = discovery_doc + def get_operations_client(self): + if not self.operations_client: + credentials = self.get_credentials() + self.operations_transport = operations_v1.OperationsRestTransport( + host="financialservices.googleapis.com", credentials=credentials + ) + self.operations_client = operations_v1.AbstractOperationsClient( + transport=self.operations_transport + ) + + return self.operations_client + def get_conn(self) -> Resource: """ Establish a connection to the Google Financial Services API. @@ -56,6 +73,14 @@ def get_conn(self) -> Resource: return self.connection + def wait_for_operation(self, operation: Operation, timeout: float | None = None): + """Wait for long-lasting operation to complete.""" + try: + return operation.result(timeout=timeout) + except Exception: + error = operation.exception(timeout=timeout) + raise AirflowException(error) + def get_instance(self, project_id: str, region: str, instance_id: str) -> dict: """ Get a Financial Services Anti-Money Laundering AI instance. @@ -74,7 +99,7 @@ def get_instance(self, project_id: str, region: str, instance_id: str) -> dict: def create_instance( self, project_id: str, region: str, instance_id: str, kms_key_ring_id: str, kms_key_id: str - ) -> dict: + ) -> Operation: """ Create a Financial Services Anti-Money Laundering AI instance. @@ -89,11 +114,13 @@ def create_instance( :returns: A dictionary containing metadata for the create instance operation """ conn = self.get_conn() + operations_client = self.get_operations_client() + parent = f"projects/{project_id}/locations/{region}" kms_key = ( f"projects/{project_id}/locations/{region}/keyRings/{kms_key_ring_id}/cryptoKeys{kms_key_id}" ) - response = ( + operation_json = ( conn.projects() .locations() .instances() @@ -104,9 +131,16 @@ def create_instance( ) .execute() ) - return response + return Operation( + operation=operation_json, + refresh=lambda: operations_client.get_operation(operation_json["name"]), + cancel=lambda: operations_client.cancel_operation(operation_json["name"]), + result_type=Empty, + # metadata_type=OperationMetadata, + polling=polling.DEFAULT_POLLING.with_timeout(timeout=21600), + ) - def delete_instance(self, project_id: str, region: str, instance_id: str) -> dict: + def delete_instance(self, project_id: str, region: str, instance_id: str) -> Operation: """ Delete a Financial Services Anti-Money Laundering AI instance. @@ -119,9 +153,19 @@ def delete_instance(self, project_id: str, region: str, instance_id: str) -> dic operation """ conn = self.get_conn() + operations_client = self.get_operations_client() + name = f"projects/{project_id}/locations/{region}/instances/{instance_id}" - response = conn.projects().locations().instances().delete(name=name).execute() - return response + operation_json = conn.projects().locations().instances().delete(name=name).execute() + + return Operation( + operation=operation_json, + refresh=lambda: operations_client.get_operation(operation_json["name"]), + cancel=lambda: operations_client.cancel_operation(operation_json["name"]), + result_type=Empty, + # metadata_type=OperationMetadata, + polling=polling.DEFAULT_POLLING.with_timeout(timeout=21600), + ) def get_operation(self, project_id: str, region: str, operation_id: str) -> dict: """ diff --git a/providers/src/airflow/providers/google/cloud/operators/financial_services.py b/providers/src/airflow/providers/google/cloud/operators/financial_services.py index 40f9220d4098..564e35c74372 100644 --- a/providers/src/airflow/providers/google/cloud/operators/financial_services.py +++ b/providers/src/airflow/providers/google/cloud/operators/financial_services.py @@ -18,6 +18,8 @@ from typing import TYPE_CHECKING, Sequence +from google.api_core.exceptions import AlreadyExists + from airflow.providers.google.cloud.hooks.financial_services import FinancialServicesHook from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator @@ -67,6 +69,7 @@ def __init__( kms_key_id: str, discovery_doc: dict, gcp_conn_id: str = "google_cloud_default", + timeout: float = 60.0, **kwargs, ) -> None: super().__init__(**kwargs) @@ -77,6 +80,7 @@ def __init__( self.kms_key_id = kms_key_id self.discovery_doc = discovery_doc self.gcp_conn_id = gcp_conn_id + self.timeout = timeout def execute(self, context: Context): hook = FinancialServicesHook( @@ -85,14 +89,23 @@ def execute(self, context: Context): ) self.log.info("Creating Financial Services instance: %s", self.instance_id) - response = hook.create_instance( - project_id=self.project_id, - region=self.region, - instance_id=self.instance_id, - kms_key_ring_id=self.kms_key_ring_id, - kms_key_id=self.kms_key_id, - ) - return response["name"] + try: + operation = hook.create_instance( + project_id=self.project_id, + region=self.region, + instance_id=self.instance_id, + kms_key_ring_id=self.kms_key_ring_id, + kms_key_id=self.kms_key_id, + ) + instance = hook.wait_for_operation( + operation=operation, + timeout=self.timeout, + ) + except AlreadyExists: + instance = hook.get_instance( + project_id=self.project_id, region=self.region, instance_id=self.instance_id + ) + return instance class FinancialServicesDeleteInstanceOperator(GoogleCloudBaseOperator): @@ -122,6 +135,7 @@ def __init__( instance_id: str, discovery_doc: dict, gcp_conn_id: str = "google_cloud_default", + timeout: float = 60.0, **kwargs, ) -> None: super().__init__(**kwargs) @@ -130,6 +144,7 @@ def __init__( self.instance_id = instance_id self.discovery_doc = discovery_doc self.gcp_conn_id = gcp_conn_id + self.timeout = timeout def execute(self, context: Context): hook = FinancialServicesHook( @@ -138,12 +153,12 @@ def execute(self, context: Context): ) self.log.info("Deleting Financial Services instance: %s", self.instance_id) - response = hook.delete_instance( + operation = hook.delete_instance( project_id=self.project_id, region=self.region, instance_id=self.instance_id, ) - return response["name"] + hook.wait_for_operation(operation=operation, timeout=self.timeout) class FinancialServicesGetInstanceOperator(GoogleCloudBaseOperator): From 892db7586421ca4197b27850db762b320315d931 Mon Sep 17 00:00:00 2001 From: claw89 Date: Fri, 1 Nov 2024 14:51:18 +0000 Subject: [PATCH 24/28] replace operation sensor with wait_for_operation method in hook --- .../google/cloud/hooks/financial_services.py | 98 ++++++++++++------- .../cloud/operators/financial_services.py | 34 +++---- .../cloud/sensors/financial_services.py | 69 ------------- .../cloud/sensors/test_financial_services.py | 83 ---------------- .../example_financial_services.py | 47 +++------ 5 files changed, 89 insertions(+), 242 deletions(-) delete mode 100644 providers/src/airflow/providers/google/cloud/sensors/financial_services.py delete mode 100644 providers/tests/google/cloud/sensors/test_financial_services.py diff --git a/providers/src/airflow/providers/google/cloud/hooks/financial_services.py b/providers/src/airflow/providers/google/cloud/hooks/financial_services.py index 188508407479..204db3566366 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/financial_services.py +++ b/providers/src/airflow/providers/google/cloud/hooks/financial_services.py @@ -16,10 +16,13 @@ # under the License. from __future__ import annotations -from google.api_core import operations_v1 +from pathlib import PurePath + from google.api_core.future import polling from google.api_core.operation import Operation +from google.longrunning import operations_pb2 from google.protobuf.empty_pb2 import Empty +from google.protobuf.json_format import ParseDict from googleapiclient.discovery import Resource, build_from_document from airflow.exceptions import AirflowException @@ -50,18 +53,6 @@ def __init__( ) self.discovery_doc = discovery_doc - def get_operations_client(self): - if not self.operations_client: - credentials = self.get_credentials() - self.operations_transport = operations_v1.OperationsRestTransport( - host="financialservices.googleapis.com", credentials=credentials - ) - self.operations_client = operations_v1.AbstractOperationsClient( - transport=self.operations_transport - ) - - return self.operations_client - def get_conn(self) -> Resource: """ Establish a connection to the Google Financial Services API. @@ -111,16 +102,15 @@ def create_instance( use for instance encryption :param kms_key_id: Required. The ID of the Google Cloud KMS key to use for instance encryption - :returns: A dictionary containing metadata for the create instance operation + :returns: The create instance operation. """ conn = self.get_conn() - operations_client = self.get_operations_client() parent = f"projects/{project_id}/locations/{region}" kms_key = ( - f"projects/{project_id}/locations/{region}/keyRings/{kms_key_ring_id}/cryptoKeys{kms_key_id}" + f"projects/{project_id}/locations/{region}/keyRings/{kms_key_ring_id}/cryptoKeys/{kms_key_id}" ) - operation_json = ( + response = ( conn.projects() .locations() .instances() @@ -131,13 +121,15 @@ def create_instance( ) .execute() ) + operation_id, operation_proto = self._parse_operation_proto(response) + return Operation( - operation=operation_json, - refresh=lambda: operations_client.get_operation(operation_json["name"]), - cancel=lambda: operations_client.cancel_operation(operation_json["name"]), + operation=operation_proto, + refresh=lambda: self._get_operation(project_id, region, operation_id), + cancel=lambda: self._cancel_operation(project_id, region, operation_id), result_type=Empty, - # metadata_type=OperationMetadata, - polling=polling.DEFAULT_POLLING.with_timeout(timeout=21600), + metadata_type=Empty, + polling=polling.DEFAULT_POLLING, ) def delete_instance(self, project_id: str, region: str, instance_id: str) -> Operation: @@ -149,36 +141,68 @@ def delete_instance(self, project_id: str, region: str, instance_id: str) -> Ope :param instance_id: Required. The ID of the instance, which is used as the final component of the instances's name. - :returns: A dictionary containing metadata for the delete instance - operation + :returns: The delete instance operation. """ conn = self.get_conn() - operations_client = self.get_operations_client() name = f"projects/{project_id}/locations/{region}/instances/{instance_id}" - operation_json = conn.projects().locations().instances().delete(name=name).execute() + response = conn.projects().locations().instances().delete(name=name).execute() + operation_id, operation_proto = self._parse_operation_proto(response) return Operation( - operation=operation_json, - refresh=lambda: operations_client.get_operation(operation_json["name"]), - cancel=lambda: operations_client.cancel_operation(operation_json["name"]), + operation=operation_proto, + refresh=lambda: self._get_operation(project_id, region, operation_id), + cancel=lambda: self._cancel_operation(project_id, region, operation_id), result_type=Empty, - # metadata_type=OperationMetadata, - polling=polling.DEFAULT_POLLING.with_timeout(timeout=21600), + metadata_type=Empty, + polling=polling.DEFAULT_POLLING, ) - def get_operation(self, project_id: str, region: str, operation_id: str) -> dict: + def _parse_operation_proto(self, json_response: dict) -> tuple[str, operations_pb2.Operation]: + """ + Parse an operation response from a Financial Services API call using operations_pb2.Operation. + + :param json_response: Required. Long-running operation data returned from the Financial Services API in JSON format. + + :returns: Tuple containing the operation ID and a parsed operations_pb2.Operation. + """ + # Can not find message descriptor by type_url: type.googleapis.com/google.cloud.financialservices.v1.OperationMetadata + # replace operation metadata protobuf with Empty + json_response["metadata"] = {"@type": "type.googleapis.com/google.protobuf.Empty"} + + if "response" in json_response.keys(): + # Can not find message descriptor by type_url: type.googleapis.com/google.cloud.financialservices.v1.Instance + # replace instance protobuf with Empty; response can be parsed, but no instance data can be returned + json_response["response"] = {"@type": "type.googleapis.com/google.protobuf.Empty"} + + operation_proto = ParseDict(js_dict=json_response, message=operations_pb2.Operation()) + operation_id = PurePath(operation_proto.name).name + return operation_id, operation_proto + + def _get_operation(self, project_id: str, region: str, operation_id: str) -> operations_pb2.Operation: """ - Get a Financial Services Anti-Money Laundering AI operation. + Get a long-running operation. :param project_id: Required. The ID of the Google Cloud project that the service belongs to. :param region: Required. The ID of the Google Cloud region that the service belongs to. - :param operation_id: Required. The ID of the operation, which is used as the final component of the - operation's name. + :param operation_id: Required. The ID of the long-running operation. - :return: A dictionary containing metadata for the operation + :returns: The parsed operations_pb2.Operation. """ conn = self.get_conn() name = f"projects/{project_id}/locations/{region}/operations/{operation_id}" response = conn.projects().locations().operations().get(name=name).execute() - return response + _, operation_proto = self._parse_operation_proto(response) + return operation_proto + + def _cancel_operation(self, project_id: str, region: str, operation_id: str): + """ + Cancel a long-running operation. + + :param project_id: Required. The ID of the Google Cloud project that the service belongs to. + :param region: Required. The ID of the Google Cloud region that the service belongs to. + :param operation_id: Required. The ID of the long-running operation. + """ + conn = self.get_conn() + name = f"projects/{project_id}/locations/{region}/operations/{operation_id}" + conn.projects().locations().operations().cancel(name=name).execute() diff --git a/providers/src/airflow/providers/google/cloud/operators/financial_services.py b/providers/src/airflow/providers/google/cloud/operators/financial_services.py index 564e35c74372..45d259a1bb27 100644 --- a/providers/src/airflow/providers/google/cloud/operators/financial_services.py +++ b/providers/src/airflow/providers/google/cloud/operators/financial_services.py @@ -18,8 +18,6 @@ from typing import TYPE_CHECKING, Sequence -from google.api_core.exceptions import AlreadyExists - from airflow.providers.google.cloud.hooks.financial_services import FinancialServicesHook from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator @@ -69,7 +67,7 @@ def __init__( kms_key_id: str, discovery_doc: dict, gcp_conn_id: str = "google_cloud_default", - timeout: float = 60.0, + timeout: float = 43200.0, # 12hr **kwargs, ) -> None: super().__init__(**kwargs) @@ -89,23 +87,17 @@ def execute(self, context: Context): ) self.log.info("Creating Financial Services instance: %s", self.instance_id) - try: - operation = hook.create_instance( - project_id=self.project_id, - region=self.region, - instance_id=self.instance_id, - kms_key_ring_id=self.kms_key_ring_id, - kms_key_id=self.kms_key_id, - ) - instance = hook.wait_for_operation( - operation=operation, - timeout=self.timeout, - ) - except AlreadyExists: - instance = hook.get_instance( - project_id=self.project_id, region=self.region, instance_id=self.instance_id - ) - return instance + operation = hook.create_instance( + project_id=self.project_id, + region=self.region, + instance_id=self.instance_id, + kms_key_ring_id=self.kms_key_ring_id, + kms_key_id=self.kms_key_id, + ) + hook.wait_for_operation( + operation=operation, + timeout=self.timeout, + ) class FinancialServicesDeleteInstanceOperator(GoogleCloudBaseOperator): @@ -135,7 +127,7 @@ def __init__( instance_id: str, discovery_doc: dict, gcp_conn_id: str = "google_cloud_default", - timeout: float = 60.0, + timeout: float = 43200.0, # 12hr **kwargs, ) -> None: super().__init__(**kwargs) diff --git a/providers/src/airflow/providers/google/cloud/sensors/financial_services.py b/providers/src/airflow/providers/google/cloud/sensors/financial_services.py deleted file mode 100644 index 13d405e03637..000000000000 --- a/providers/src/airflow/providers/google/cloud/sensors/financial_services.py +++ /dev/null @@ -1,69 +0,0 @@ -# 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 typing import TYPE_CHECKING, Sequence - -from airflow.exceptions import AirflowFailException -from airflow.providers.google.cloud.hooks.financial_services import FinancialServicesHook -from airflow.sensors.base import BaseSensorOperator, PokeReturnValue - -if TYPE_CHECKING: - from airflow.utils.context import Context - - -class FinancialServicesOperationSensor(BaseSensorOperator): - """ - Check status of AML AI operation. - - .. seealso:: - For more information on how to use this operator, take a look at the guide: - :ref:`howto/sensor:FinancialServicesOperationSensor` - - :param operation_resource_uri: URI of the operation (format: - 'projects//locations//operations/) - :param discovery_doc: Discovery document for building the Financial Services API - as described `here `__ - :param gcp_conn_id: Identifier of connection to Google Cloud Platform. - Defaults to "google_cloud_default". - """ - - # [START howto_sensor_financial_services_operation_template_fields] - template_fields: Sequence[str] = ("operation_resource_uri", "discovery_doc", "gcp_conn_id") - # [END howto_sensor_financial_services_operation_template_fields] - - def __init__( - self, - operation_resource_uri: str, - discovery_doc: dict, - gcp_conn_id: str = "google_cloud_default", - **kwargs, - ): - super().__init__(**kwargs) - self.operation_resource_uri = operation_resource_uri - self.discovery_doc = discovery_doc - self.gcp_conn_id = gcp_conn_id - - def poke(self, context: Context) -> PokeReturnValue: - hook = FinancialServicesHook( - discovery_doc=self.discovery_doc, - gcp_conn_id=self.gcp_conn_id, - ) - operation = hook.get_operation(operation_resource_uri=self.operation_resource_uri) - if "error" in operation.keys(): - raise AirflowFailException(operation["error"]) - return PokeReturnValue(is_done=operation["done"]) diff --git a/providers/tests/google/cloud/sensors/test_financial_services.py b/providers/tests/google/cloud/sensors/test_financial_services.py deleted file mode 100644 index d115f548a11a..000000000000 --- a/providers/tests/google/cloud/sensors/test_financial_services.py +++ /dev/null @@ -1,83 +0,0 @@ -# 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 import mock - -import pytest - -from airflow.exceptions import AirflowFailException -from airflow.providers.google.cloud.sensors.financial_services import FinancialServicesOperationSensor - -TEST_LOCATION_RESOURCE_URI = "projects/test-project/locations/us-central1" -TEST_OPERATION_RESOURCE_URI = f"{TEST_LOCATION_RESOURCE_URI}/operations/test-operation" -TEST_OPERATION = {"name": "test-operation", "metadata": {}, "done": False} -TEST_OPERATION_DONE = {"name": "test-operation", "metadata": {}, "done": True, "response": {}} -TEST_OPERATION_ERROR = {"name": "test-operation", "metadata": {}, "done": True, "error": {}} - - -class TestFinancialServicesOperationSensor: - @mock.patch("airflow.providers.google.cloud.sensors.financial_services.FinancialServicesHook") - def test_poke(self, mock_hook): - mock_hook.return_value.get_operation.return_value = TEST_OPERATION - - op = FinancialServicesOperationSensor( - task_id="test_operation_sensor_task", - discovery_doc={}, - operation_resource_uri=TEST_OPERATION_RESOURCE_URI, - ) - response = op.poke(context={"ti": mock.MagicMock()}) - - mock_hook.assert_called_once_with(discovery_doc={}, gcp_conn_id="google_cloud_default") - mock_hook.return_value.get_operation.assert_called_once_with( - operation_resource_uri=TEST_OPERATION_RESOURCE_URI, - ) - assert response.is_done == TEST_OPERATION["done"] - - @mock.patch("airflow.providers.google.cloud.sensors.financial_services.FinancialServicesHook") - def test_poke_done(self, mock_hook): - mock_hook.return_value.get_operation.return_value = TEST_OPERATION_DONE - - op = FinancialServicesOperationSensor( - task_id="test_operation_sensor_task", - discovery_doc={}, - operation_resource_uri=TEST_OPERATION_RESOURCE_URI, - ) - response = op.poke(context={"ti": mock.MagicMock()}) - - mock_hook.assert_called_once_with(discovery_doc={}, gcp_conn_id="google_cloud_default") - mock_hook.return_value.get_operation.assert_called_once_with( - operation_resource_uri=TEST_OPERATION_RESOURCE_URI, - ) - assert response.is_done == TEST_OPERATION_DONE["done"] - - @mock.patch("airflow.providers.google.cloud.sensors.financial_services.FinancialServicesHook") - def test_poke_error(self, mock_hook): - mock_hook.return_value.get_operation.return_value = TEST_OPERATION_ERROR - - op = FinancialServicesOperationSensor( - task_id="test_operation_sensor_task", - discovery_doc={}, - operation_resource_uri=TEST_OPERATION_RESOURCE_URI, - ) - with pytest.raises(AirflowFailException): - op.poke(context={"ti": mock.MagicMock()}) - - mock_hook.assert_called_once_with(discovery_doc={}, gcp_conn_id="google_cloud_default") - mock_hook.return_value.get_operation.assert_called_once_with( - operation_resource_uri=TEST_OPERATION_RESOURCE_URI, - ) diff --git a/providers/tests/system/google/cloud/financial_services/example_financial_services.py b/providers/tests/system/google/cloud/financial_services/example_financial_services.py index fbdd928c5002..63ed652c21b0 100644 --- a/providers/tests/system/google/cloud/financial_services/example_financial_services.py +++ b/providers/tests/system/google/cloud/financial_services/example_financial_services.py @@ -18,7 +18,7 @@ import json import os -from datetime import datetime, timedelta +from datetime import datetime from pathlib import Path from airflow.decorators import task @@ -28,14 +28,16 @@ FinancialServicesDeleteInstanceOperator, FinancialServicesGetInstanceOperator, ) -from airflow.providers.google.cloud.sensors.financial_services import FinancialServicesOperationSensor +# from airflow.providers.google.cloud.sensors.financial_services import FinancialServicesOperationSensor from providers.tests.system.google import DEFAULT_GCP_SYSTEM_TEST_PROJECT_ID ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID", "default") PROJECT_ID = os.environ.get("SYSTEM_TESTS_GCP_PROJECT") or DEFAULT_GCP_SYSTEM_TEST_PROJECT_ID LOCATION = os.environ.get("SYSTEM_TESTS_GCP_LOCATION", "us-central1") +KMS_KEY_RING = os.environ.get("SYSTEM_TESTS_GCP_KMS_KEY_RING") KMS_KEY = os.environ.get("SYSTEM_TESTS_GCP_KMS_KEY") +assert KMS_KEY_RING is not None assert KMS_KEY is not None DAG_ID = "financial_services_instance" @@ -61,27 +63,21 @@ def load_discovery_doc() -> str: create_instance_task = FinancialServicesCreateInstanceOperator( task_id="create_instance_task", discovery_doc=discovery_doc, + project_id=PROJECT_ID, + region=LOCATION, instance_id=INSTANCE_ID, - location_resource_uri=f"projects/{PROJECT_ID}/locations/{LOCATION}", - kms_key_uri=KMS_KEY, + kms_key_ring_id=KMS_KEY_RING, + kms_key_id=KMS_KEY, ) # [END howto_operator_financial_services_create_instance] - # [START howto_sensor_financial_services_operation] - create_instance_sensor = FinancialServicesOperationSensor( - task_id="create_instance_sensor", - discovery_doc=discovery_doc, - operation_resource_uri="{{ task_instance.xcom_pull(task_ids='create_instance_task', key='return_value') }}", - poke_interval=timedelta(seconds=5), - timeout=timedelta(hours=1), - ) - # [END howto_sensor_financial_services_operation] - # [START howto_operator_financial_services_get_instance] get_instance_task = FinancialServicesGetInstanceOperator( task_id="get_instance_task", discovery_doc=discovery_doc, - instance_resource_uri=f"projects/{PROJECT_ID}/locations/{LOCATION}/instances/{INSTANCE_ID}", + project_id=PROJECT_ID, + region=LOCATION, + instance_id=INSTANCE_ID, ) # [END howto_operator_financial_services_get_instance] @@ -89,26 +85,13 @@ def load_discovery_doc() -> str: delete_instance_task = FinancialServicesDeleteInstanceOperator( task_id="delete_instance_task", discovery_doc=discovery_doc, - instance_resource_uri=f"projects/{PROJECT_ID}/locations/{LOCATION}/instances/{INSTANCE_ID}", + project_id=PROJECT_ID, + region=LOCATION, + instance_id=INSTANCE_ID, ) # [END howto_operator_financial_services_delete_instance] - delete_instance_sensor = FinancialServicesOperationSensor( - task_id="delete_instance_sensor", - discovery_doc=discovery_doc, - operation_resource_uri="{{ task_instance.xcom_pull(task_ids='delete_instance_task', key='return_value') }}", - poke_interval=timedelta(seconds=5), - timeout=timedelta(hours=1), - ) - - ( - discovery_doc - >> create_instance_task - >> create_instance_sensor - >> get_instance_task - >> delete_instance_task - >> delete_instance_sensor - ) + (discovery_doc >> create_instance_task >> get_instance_task >> delete_instance_task) from tests_common.test_utils.watcher import watcher From 1058c09072102c7ab9dec369e01e1ec7edba0545 Mon Sep 17 00:00:00 2001 From: claw89 Date: Fri, 1 Nov 2024 15:13:45 +0000 Subject: [PATCH 25/28] update test args --- .../cloud/hooks/test_financial_services.py | 62 ++++++++----------- .../operators/test_financial_services.py | 56 ++++++++--------- 2 files changed, 54 insertions(+), 64 deletions(-) diff --git a/providers/tests/google/cloud/hooks/test_financial_services.py b/providers/tests/google/cloud/hooks/test_financial_services.py index f8a577e16b91..c7dcd1bb8da2 100644 --- a/providers/tests/google/cloud/hooks/test_financial_services.py +++ b/providers/tests/google/cloud/hooks/test_financial_services.py @@ -20,19 +20,11 @@ from airflow.providers.google.cloud.hooks.financial_services import FinancialServicesHook -TEST_KMS_KEY_URI = "projects/test-project/locations/us-central1/keyRings/my-kr/cryptoKeys/my-kms-key" -TEST_LOCATION_RESOURCE_URI = "projects/test-project/locations/us-central1" -TEST_INSTANCE_ID = "test-instance" -TEST_INSTANCE_RESOURCE_URI = f"{TEST_LOCATION_RESOURCE_URI}/instances/{TEST_INSTANCE_ID}" -TEST_OPERATION = {"name": "test-operation", "metadata": {}, "done": False} -TEST_INSTANCE = { - "name": "test-instance", - "createTime": "2014-10-02T15:01:23Z", - "updateTime": "2014-10-02T15:01:23Z", - "labels": {}, - "state": "ACTIVE", - "kmsKey": TEST_KMS_KEY_URI, -} +PROJECT_ID = "test-project" +REGION = "us-central1" +KMS_KEY_RING = "test-key-ring" +KMS_KEY = "test-key" +INSTANCE_ID = "test-instance" def mock_init( @@ -66,13 +58,13 @@ def test_get_instance(self, mock_get_conn): projects = self.financial_services_hook.connection.projects.return_value locations = projects.locations.return_value instances = locations.instances.return_value - instances.get.return_value.execute.return_value = TEST_INSTANCE - response = self.financial_services_hook.get_instance(instance_resource_uri=TEST_INSTANCE_RESOURCE_URI) - - instances.get.assert_called_once_with(name=TEST_INSTANCE_RESOURCE_URI) - - assert response == TEST_INSTANCE + self.financial_services_hook.get_instance( + project_id=PROJECT_ID, region=REGION, instance_id=INSTANCE_ID + ) + instances.get.assert_called_once_with( + name=f"projects/{PROJECT_ID}/locations/{REGION}/instances/{INSTANCE_ID}" + ) @patch("airflow.providers.google.cloud.hooks.financial_services.FinancialServicesHook.get_conn") def test_create_instance(self, mock_get_conn): @@ -81,22 +73,23 @@ def test_create_instance(self, mock_get_conn): projects = self.financial_services_hook.connection.projects.return_value locations = projects.locations.return_value instances = locations.instances.return_value - instances.create.return_value.execute.return_value = TEST_OPERATION - response = self.financial_services_hook.create_instance( - instance_id=TEST_INSTANCE_ID, - kms_key_uri=TEST_KMS_KEY_URI, - location_resource_uri=TEST_LOCATION_RESOURCE_URI, + self.financial_services_hook.create_instance( + project_id=PROJECT_ID, + region=REGION, + instance_id=INSTANCE_ID, + kms_key_ring_id=KMS_KEY_RING, + kms_key_id=KMS_KEY, ) instances.create.assert_called_once_with( - parent=TEST_LOCATION_RESOURCE_URI, - instanceId=TEST_INSTANCE_ID, - body={"kmsKey": TEST_KMS_KEY_URI}, + parent=f"projects/{PROJECT_ID}/locations/{REGION}", + instanceId=INSTANCE_ID, + body={ + "kmsKey": f"projects/{PROJECT_ID}/locations/{REGION}/keyRings/{KMS_KEY_RING}/cryptoKeys/{KMS_KEY}" + }, ) - assert response == TEST_OPERATION - @patch("airflow.providers.google.cloud.hooks.financial_services.FinancialServicesHook.get_conn") def test_delete_instance(self, mock_get_conn): self.financial_services_hook.connection = mock_get_conn.return_value @@ -104,12 +97,11 @@ def test_delete_instance(self, mock_get_conn): projects = self.financial_services_hook.connection.projects.return_value locations = projects.locations.return_value instances = locations.instances.return_value - instances.delete.return_value.execute.return_value = TEST_OPERATION - response = self.financial_services_hook.delete_instance( - instance_resource_uri=TEST_INSTANCE_RESOURCE_URI + self.financial_services_hook.delete_instance( + project_id=PROJECT_ID, region=REGION, instance_id=INSTANCE_ID ) - instances.delete.assert_called_once_with(name=TEST_INSTANCE_RESOURCE_URI) - - assert response == TEST_OPERATION + instances.delete.assert_called_once_with( + name=f"projects/{PROJECT_ID}/locations/{REGION}/instances/{INSTANCE_ID}" + ) diff --git a/providers/tests/google/cloud/operators/test_financial_services.py b/providers/tests/google/cloud/operators/test_financial_services.py index 2d54d91fbb4a..99649de0c9f8 100644 --- a/providers/tests/google/cloud/operators/test_financial_services.py +++ b/providers/tests/google/cloud/operators/test_financial_services.py @@ -23,74 +23,72 @@ FinancialServicesDeleteInstanceOperator, ) -TEST_KMS_KEY_URI = "projects/test-project/locations/us-central1/keyRings/my-kr/cryptoKeys/my-kms-key" -TEST_LOCATION_RESOURCE_URI = "projects/test-project/locations/us-central1" -TEST_INSTANCE_ID = "test-instance" -TEST_INSTANCE_RESOURCE_URI = f"{TEST_LOCATION_RESOURCE_URI}/instances/{TEST_INSTANCE_ID}" -TEST_OPERATION = {"name": "test-operation", "metadata": {}, "done": False} -TEST_INSTANCE = { - "name": "test-instance", - "createTime": "2014-10-02T15:01:23Z", - "updateTime": "2014-10-02T15:01:23Z", - "labels": {}, - "state": "ACTIVE", - "kmsKey": TEST_KMS_KEY_URI, -} +PROJECT_ID = "test-project" +REGION = "us-central1" +KMS_KEY_RING = "test-key-ring" +KMS_KEY = "test-key" +INSTANCE_ID = "test-instance" class TestFinancialServicesCreateInstanceOperator: @mock.patch("airflow.providers.google.cloud.operators.financial_services.FinancialServicesHook") def test_execute(self, mock_hook): - mock_hook.return_value.create_instance.return_value = TEST_OPERATION - op = FinancialServicesCreateInstanceOperator( task_id="test_create_instance_task", + project_id=PROJECT_ID, + region=REGION, + instance_id=INSTANCE_ID, + kms_key_ring_id=KMS_KEY_RING, + kms_key_id=KMS_KEY, discovery_doc={}, - instance_id=TEST_INSTANCE_ID, - kms_key_uri=TEST_KMS_KEY_URI, - location_resource_uri=TEST_LOCATION_RESOURCE_URI, ) op.execute(context={"ti": mock.MagicMock()}) mock_hook.assert_called_once_with(discovery_doc={}, gcp_conn_id="google_cloud_default") mock_hook.return_value.create_instance.assert_called_once_with( - instance_id=TEST_INSTANCE_ID, - kms_key_uri=TEST_KMS_KEY_URI, - location_resource_uri=TEST_LOCATION_RESOURCE_URI, + project_id=PROJECT_ID, + region=REGION, + instance_id=INSTANCE_ID, + kms_key_ring_id=KMS_KEY_RING, + kms_key_id=KMS_KEY, ) class TestFinancialServicesDeleteInstanceOperator: @mock.patch("airflow.providers.google.cloud.operators.financial_services.FinancialServicesHook") def test_execute(self, mock_hook): - mock_hook.return_value.delete_instance.return_value = TEST_OPERATION - op = FinancialServicesDeleteInstanceOperator( task_id="test_delete_instance_task", + project_id=PROJECT_ID, + region=REGION, + instance_id=INSTANCE_ID, discovery_doc={}, - instance_resource_uri=TEST_INSTANCE_RESOURCE_URI, ) op.execute(context={"ti": mock.MagicMock()}) mock_hook.assert_called_once_with(discovery_doc={}, gcp_conn_id="google_cloud_default") mock_hook.return_value.delete_instance.assert_called_once_with( - instance_resource_uri=TEST_INSTANCE_RESOURCE_URI + project_id=PROJECT_ID, + region=REGION, + instance_id=INSTANCE_ID, ) class TestFinancialServicesGetInstanceOperator: @mock.patch("airflow.providers.google.cloud.operators.financial_services.FinancialServicesHook") def test_execute(self, mock_hook): - mock_hook.return_value.get_instance.return_value = TEST_INSTANCE - op = FinancialServicesDeleteInstanceOperator( task_id="test_get_instance_task", + project_id=PROJECT_ID, + region=REGION, + instance_id=INSTANCE_ID, discovery_doc={}, - instance_resource_uri=TEST_INSTANCE_RESOURCE_URI, ) op.execute(context={"ti": mock.MagicMock()}) mock_hook.assert_called_once_with(discovery_doc={}, gcp_conn_id="google_cloud_default") mock_hook.return_value.delete_instance.assert_called_once_with( - instance_resource_uri=TEST_INSTANCE_RESOURCE_URI + project_id=PROJECT_ID, + region=REGION, + instance_id=INSTANCE_ID, ) From fb67fd9b3dc6a7a4c8b8b06ea81c1416f3c5e881 Mon Sep 17 00:00:00 2001 From: claw89 Date: Fri, 1 Nov 2024 16:17:25 +0000 Subject: [PATCH 26/28] add developer key option for connecting to financial serivces --- .../google/cloud/hooks/financial_services.py | 39 +++++++++++++++++-- .../cloud/operators/financial_services.py | 14 +------ .../example_financial_services.py | 16 +------- 3 files changed, 38 insertions(+), 31 deletions(-) diff --git a/providers/src/airflow/providers/google/cloud/hooks/financial_services.py b/providers/src/airflow/providers/google/cloud/hooks/financial_services.py index 204db3566366..2558e01a307b 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/financial_services.py +++ b/providers/src/airflow/providers/google/cloud/hooks/financial_services.py @@ -16,6 +16,7 @@ # under the License. from __future__ import annotations +import json from pathlib import PurePath from google.api_core.future import polling @@ -23,9 +24,10 @@ from google.longrunning import operations_pb2 from google.protobuf.empty_pb2 import Empty from google.protobuf.json_format import ParseDict -from googleapiclient.discovery import Resource, build_from_document +from googleapiclient.discovery import Resource, build, build_from_document from airflow.exceptions import AirflowException +from airflow.models import Variable from airflow.providers.google.common.hooks.base_google import GoogleBaseHook @@ -43,7 +45,6 @@ class FinancialServicesHook(GoogleBaseHook): def __init__( self, - discovery_doc: dict, gcp_conn_id: str = "google_cloud_default", **kwargs, ) -> None: @@ -51,7 +52,18 @@ def __init__( gcp_conn_id=gcp_conn_id, impersonation_chain=None, ) - self.discovery_doc = discovery_doc + + def _get_developer_key(self) -> str | None: + return Variable.get("financial_services_api_key", default_var=None) + + def _get_discovery_doc(self) -> dict | None: + discovery_doc_path = Variable.get("financial_services_discovery_doc_path", default_var=None) + if not discovery_doc_path: + discovery_doc = None + else: + with open(discovery_doc_path) as file: + discovery_doc = json.load(file) + return discovery_doc def get_conn(self) -> Resource: """ @@ -60,7 +72,26 @@ def get_conn(self) -> Resource: :return: A Google Cloud Financial Services API service resource. """ if not self.connection: - self.connection = build_from_document(self.discovery_doc) + developer_key = self._get_developer_key() + discovery_doc = self._get_discovery_doc() + + if developer_key: + credentials = self.get_credentials() + self.connection = build( + serviceName="financialservices", + version="v1", + developerKey=developer_key, + discoveryServiceUrl="https://financialservices.googleapis.com/$discovery/rest?version=v1", + credentials=credentials, + ) + elif discovery_doc: + self.connection = build_from_document(discovery_doc) + else: + raise AirflowException( + "Connecting to financialservices.googleapis.com requires either 'financial_services_api_key' or " + "'financial_services_discovery_doc_path' to be set in Airflow variables. Use 'airflow variables set financial_services_api_key ' " + "to set the variable" + ) return self.connection diff --git a/providers/src/airflow/providers/google/cloud/operators/financial_services.py b/providers/src/airflow/providers/google/cloud/operators/financial_services.py index 45d259a1bb27..f9593178e39a 100644 --- a/providers/src/airflow/providers/google/cloud/operators/financial_services.py +++ b/providers/src/airflow/providers/google/cloud/operators/financial_services.py @@ -53,7 +53,6 @@ class FinancialServicesCreateInstanceOperator(GoogleCloudBaseOperator): "instance_id", "kms_key_ring_id", "kms_key_id", - "discovery_doc", "gcp_conn_id", ) # [END howto_operator_financial_services_create_instance_template_fields] @@ -65,7 +64,6 @@ def __init__( instance_id: str, kms_key_ring_id: str, kms_key_id: str, - discovery_doc: dict, gcp_conn_id: str = "google_cloud_default", timeout: float = 43200.0, # 12hr **kwargs, @@ -76,13 +74,11 @@ def __init__( self.instance_id = instance_id self.kms_key_ring_id = kms_key_ring_id self.kms_key_id = kms_key_id - self.discovery_doc = discovery_doc self.gcp_conn_id = gcp_conn_id self.timeout = timeout def execute(self, context: Context): hook = FinancialServicesHook( - discovery_doc=self.discovery_doc, gcp_conn_id=self.gcp_conn_id, ) self.log.info("Creating Financial Services instance: %s", self.instance_id) @@ -117,7 +113,7 @@ class FinancialServicesDeleteInstanceOperator(GoogleCloudBaseOperator): """ # [START howto_operator_financial_services_get_instance_template_fields] - template_fields: Sequence[str] = ("project_id", "region", "instance_id", "discovery_doc", "gcp_conn_id") + template_fields: Sequence[str] = ("project_id", "region", "instance_id", "gcp_conn_id") # [END howto_operator_financial_services_get_instance_template_fields] def __init__( @@ -125,7 +121,6 @@ def __init__( project_id: str, region: str, instance_id: str, - discovery_doc: dict, gcp_conn_id: str = "google_cloud_default", timeout: float = 43200.0, # 12hr **kwargs, @@ -134,13 +129,11 @@ def __init__( self.project_id = project_id self.region = region self.instance_id = instance_id - self.discovery_doc = discovery_doc self.gcp_conn_id = gcp_conn_id self.timeout = timeout def execute(self, context: Context): hook = FinancialServicesHook( - discovery_doc=self.discovery_doc, gcp_conn_id=self.gcp_conn_id, ) self.log.info("Deleting Financial Services instance: %s", self.instance_id) @@ -170,7 +163,7 @@ class FinancialServicesGetInstanceOperator(GoogleCloudBaseOperator): """ # [START howto_operator_financial_services_delete_instance_template_fields] - template_fields: Sequence[str] = ("project_id", "region", "instance_id", "discovery_doc", "gcp_conn_id") + template_fields: Sequence[str] = ("project_id", "region", "instance_id", "gcp_conn_id") # [END howto_operator_financial_services_delete_instance_template_fields] def __init__( @@ -178,7 +171,6 @@ def __init__( project_id: str, region: str, instance_id: str, - discovery_doc: dict, gcp_conn_id: str = "google_cloud_default", **kwargs, ) -> None: @@ -186,12 +178,10 @@ def __init__( self.project_id = project_id self.region = region self.instance_id = instance_id - self.discovery_doc = discovery_doc self.gcp_conn_id = gcp_conn_id def execute(self, context: Context): hook = FinancialServicesHook( - discovery_doc=self.discovery_doc, gcp_conn_id=self.gcp_conn_id, ) self.log.info("Fetching Financial Services instance: %s", self.instance_id) diff --git a/providers/tests/system/google/cloud/financial_services/example_financial_services.py b/providers/tests/system/google/cloud/financial_services/example_financial_services.py index 63ed652c21b0..fa712e54d786 100644 --- a/providers/tests/system/google/cloud/financial_services/example_financial_services.py +++ b/providers/tests/system/google/cloud/financial_services/example_financial_services.py @@ -16,12 +16,10 @@ # under the License. from __future__ import annotations -import json import os from datetime import datetime from pathlib import Path -from airflow.decorators import task from airflow.models.dag import DAG from airflow.providers.google.cloud.operators.financial_services import ( FinancialServicesCreateInstanceOperator, @@ -50,19 +48,9 @@ start_date=datetime(2021, 1, 1), catchup=False, ) as dag: - - @task() - def load_discovery_doc() -> str: - with open(RESOURCE_DIR_PATH) as file: - doc = json.load(file) - return doc - - discovery_doc = load_discovery_doc() - # [START howto_operator_financial_services_create_instance] create_instance_task = FinancialServicesCreateInstanceOperator( task_id="create_instance_task", - discovery_doc=discovery_doc, project_id=PROJECT_ID, region=LOCATION, instance_id=INSTANCE_ID, @@ -74,7 +62,6 @@ def load_discovery_doc() -> str: # [START howto_operator_financial_services_get_instance] get_instance_task = FinancialServicesGetInstanceOperator( task_id="get_instance_task", - discovery_doc=discovery_doc, project_id=PROJECT_ID, region=LOCATION, instance_id=INSTANCE_ID, @@ -84,14 +71,13 @@ def load_discovery_doc() -> str: # [START howto_operator_financial_services_delete_instance] delete_instance_task = FinancialServicesDeleteInstanceOperator( task_id="delete_instance_task", - discovery_doc=discovery_doc, project_id=PROJECT_ID, region=LOCATION, instance_id=INSTANCE_ID, ) # [END howto_operator_financial_services_delete_instance] - (discovery_doc >> create_instance_task >> get_instance_task >> delete_instance_task) + (create_instance_task >> get_instance_task >> delete_instance_task) from tests_common.test_utils.watcher import watcher From e43f6e590dc68a2870dfc853bbfdd8ae0a6b7091 Mon Sep 17 00:00:00 2001 From: claw89 Date: Fri, 1 Nov 2024 16:36:43 +0000 Subject: [PATCH 27/28] update unit tests for new connection options --- .../cloud/hooks/test_financial_services.py | 19 +++++++++++++++++-- .../operators/test_financial_services.py | 9 +++------ 2 files changed, 20 insertions(+), 8 deletions(-) diff --git a/providers/tests/google/cloud/hooks/test_financial_services.py b/providers/tests/google/cloud/hooks/test_financial_services.py index c7dcd1bb8da2..e0c7c39fa5a1 100644 --- a/providers/tests/google/cloud/hooks/test_financial_services.py +++ b/providers/tests/google/cloud/hooks/test_financial_services.py @@ -26,6 +26,8 @@ KMS_KEY = "test-key" INSTANCE_ID = "test-instance" +API_KEY = "*********************" + def mock_init( self, @@ -41,10 +43,23 @@ def setup_method(self): "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.__init__", new=mock_init, ): - self.financial_services_hook = FinancialServicesHook(discovery_doc={}) + self.financial_services_hook = FinancialServicesHook() + + @patch("airflow.providers.google.cloud.hooks.financial_services.FinancialServicesHook.get_credentials") + @patch("airflow.providers.google.cloud.hooks.financial_services.FinancialServicesHook._get_developer_key") + @patch("airflow.providers.google.cloud.hooks.financial_services.build") + def test_get_conn_with_api_key(self, mock_build, mock_developer_key, mock_get_credentials): + mock_developer_key.return_value = API_KEY + conn = self.financial_services_hook.get_conn() + + mock_build.assert_called_once() + assert conn == mock_build.return_value + assert conn == self.financial_services_hook.connection + @patch("airflow.providers.google.cloud.hooks.financial_services.FinancialServicesHook._get_discovery_doc") @patch("airflow.providers.google.cloud.hooks.financial_services.build_from_document") - def test_get_conn(self, mock_build): + def test_get_conn_with_discovery_doc(self, mock_build, mock_discovery_doc): + mock_discovery_doc.return_value = {"schemas": {}} conn = self.financial_services_hook.get_conn() mock_build.assert_called_once() diff --git a/providers/tests/google/cloud/operators/test_financial_services.py b/providers/tests/google/cloud/operators/test_financial_services.py index 99649de0c9f8..277df9fd3d32 100644 --- a/providers/tests/google/cloud/operators/test_financial_services.py +++ b/providers/tests/google/cloud/operators/test_financial_services.py @@ -40,11 +40,10 @@ def test_execute(self, mock_hook): instance_id=INSTANCE_ID, kms_key_ring_id=KMS_KEY_RING, kms_key_id=KMS_KEY, - discovery_doc={}, ) op.execute(context={"ti": mock.MagicMock()}) - mock_hook.assert_called_once_with(discovery_doc={}, gcp_conn_id="google_cloud_default") + mock_hook.assert_called_once_with(gcp_conn_id="google_cloud_default") mock_hook.return_value.create_instance.assert_called_once_with( project_id=PROJECT_ID, region=REGION, @@ -62,11 +61,10 @@ def test_execute(self, mock_hook): project_id=PROJECT_ID, region=REGION, instance_id=INSTANCE_ID, - discovery_doc={}, ) op.execute(context={"ti": mock.MagicMock()}) - mock_hook.assert_called_once_with(discovery_doc={}, gcp_conn_id="google_cloud_default") + mock_hook.assert_called_once_with(gcp_conn_id="google_cloud_default") mock_hook.return_value.delete_instance.assert_called_once_with( project_id=PROJECT_ID, region=REGION, @@ -82,11 +80,10 @@ def test_execute(self, mock_hook): project_id=PROJECT_ID, region=REGION, instance_id=INSTANCE_ID, - discovery_doc={}, ) op.execute(context={"ti": mock.MagicMock()}) - mock_hook.assert_called_once_with(discovery_doc={}, gcp_conn_id="google_cloud_default") + mock_hook.assert_called_once_with(gcp_conn_id="google_cloud_default") mock_hook.return_value.delete_instance.assert_called_once_with( project_id=PROJECT_ID, region=REGION, From cc783feaacb9cfad648a2e2a79b0d99d7543f9d1 Mon Sep 17 00:00:00 2001 From: claw89 Date: Fri, 1 Nov 2024 18:44:15 +0000 Subject: [PATCH 28/28] update documentation --- .../operators/cloud/financial_services.rst | 37 ------------------- .../google/cloud/hooks/financial_services.py | 29 +++++++++++---- .../cloud/operators/financial_services.py | 20 +++++----- .../airflow/providers/google/provider.yaml | 3 -- 4 files changed, 32 insertions(+), 57 deletions(-) diff --git a/docs/apache-airflow-providers-google/operators/cloud/financial_services.rst b/docs/apache-airflow-providers-google/operators/cloud/financial_services.rst index 367e5f3871de..4c1471bc82a9 100644 --- a/docs/apache-airflow-providers-google/operators/cloud/financial_services.rst +++ b/docs/apache-airflow-providers-google/operators/cloud/financial_services.rst @@ -127,40 +127,3 @@ More information See Google Cloud Financial Services API documentation to `delete an instance `_. - - - -Sensors -^^^^^^^ - -.. _howto/sensor:FinancialServicesOperationSensor: - -Check operation status ----------------------- - -Use the :class:`~airflow.providers.google.cloud.sensors.financial_services.FinancialServicesOperationSensor` -sensor to check the status of an operation in the Google Cloud Financial Services API. - -Using the sensor -"""""""""""""""""" - -.. exampleinclude:: /../../providers/tests/system/google/cloud/financial_services/example_financial_services.py - :language: python - :dedent: 4 - :start-after: [START howto_sensor_financial_services_operation] - :end-before: [END howto_sensor_financial_services_operation] - -Templating -"""""""""" - -.. literalinclude:: /../../providers/src/airflow/providers/google/cloud/sensors/financial_services.py - :language: python - :dedent: 4 - :start-after: [START howto_sensor_financial_services_operation_template_fields] - :end-before: [END howto_sensor_financial_services_operation_template_fields] - -More information -"""""""""""""""" - -See Google Cloud Financial Services API documentation to `get an operation -`_. diff --git a/providers/src/airflow/providers/google/cloud/hooks/financial_services.py b/providers/src/airflow/providers/google/cloud/hooks/financial_services.py index 2558e01a307b..b3b3963df7ca 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/financial_services.py +++ b/providers/src/airflow/providers/google/cloud/hooks/financial_services.py @@ -35,8 +35,6 @@ class FinancialServicesHook(GoogleBaseHook): """ Hook for interacting with the Google Financial Services API. - :param discovery_doc: Discovery document for building the Financial Services API - as described `here `__ :param gcp_conn_id: Identifier of connection to Google Cloud Platform. Defaults to "google_cloud_default". """ @@ -54,9 +52,19 @@ def __init__( ) def _get_developer_key(self) -> str | None: + """ + Get a developer key for accessing the Financial Services discovery API. + + :return: A developer key for discovery API access. + """ return Variable.get("financial_services_api_key", default_var=None) def _get_discovery_doc(self) -> dict | None: + """ + Get the Financial Services discovery document from local storage. + + :return: Financial Services discovery document. + """ discovery_doc_path = Variable.get("financial_services_discovery_doc_path", default_var=None) if not discovery_doc_path: discovery_doc = None @@ -89,14 +97,21 @@ def get_conn(self) -> Resource: else: raise AirflowException( "Connecting to financialservices.googleapis.com requires either 'financial_services_api_key' or " - "'financial_services_discovery_doc_path' to be set in Airflow variables. Use 'airflow variables set financial_services_api_key ' " - "to set the variable" + "'financial_services_discovery_doc_path' to be set in Airflow variables. Use 'airflow variables " + "set financial_services_api_key ' to set the variable" ) return self.connection def wait_for_operation(self, operation: Operation, timeout: float | None = None): - """Wait for long-lasting operation to complete.""" + """ + Wait for the completion of a long-running operation. + + :param operation: The long-running operation to wait for completion. + :param timeout: Timeout in seconds to wait for completion. + + :return: The result of the operation after completion. + """ try: return operation.result(timeout=timeout) except Exception: @@ -195,7 +210,7 @@ def _parse_operation_proto(self, json_response: dict) -> tuple[str, operations_p :param json_response: Required. Long-running operation data returned from the Financial Services API in JSON format. - :returns: Tuple containing the operation ID and a parsed operations_pb2.Operation. + :returns: The operation ID and a parsed pb2 operation. """ # Can not find message descriptor by type_url: type.googleapis.com/google.cloud.financialservices.v1.OperationMetadata # replace operation metadata protobuf with Empty @@ -218,7 +233,7 @@ def _get_operation(self, project_id: str, region: str, operation_id: str) -> ope :param region: Required. The ID of the Google Cloud region that the service belongs to. :param operation_id: Required. The ID of the long-running operation. - :returns: The parsed operations_pb2.Operation. + :returns: The parsed pb2 operation. """ conn = self.get_conn() name = f"projects/{project_id}/locations/{region}/operations/{operation_id}" diff --git a/providers/src/airflow/providers/google/cloud/operators/financial_services.py b/providers/src/airflow/providers/google/cloud/operators/financial_services.py index f9593178e39a..ce0fb181223d 100644 --- a/providers/src/airflow/providers/google/cloud/operators/financial_services.py +++ b/providers/src/airflow/providers/google/cloud/operators/financial_services.py @@ -40,10 +40,9 @@ class FinancialServicesCreateInstanceOperator(GoogleCloudBaseOperator): :param kms_key_ring_id: Required. The ID of the Google Cloud KMS key ring containing the key to use for instance encryption :param kms_key_id: Required. The ID of the Google Cloud KMS key to use for instance encryption - :param discovery_doc: Discovery document for building the Financial Services API - as described `here `__ :param gcp_conn_id: Identifier of connection to Google Cloud Platform. Defaults to "google_cloud_default". + :param timeout: The maximum time to wait for a long-running operation to complete. """ # [START howto_operator_financial_services_create_instance_template_fields] @@ -104,12 +103,13 @@ class FinancialServicesDeleteInstanceOperator(GoogleCloudBaseOperator): For more information on how to use this operator, take a look at the guide: :ref:`howto/operator:FinancialServicesDeleteInstanceOperator` - :param instance_resource_uri: URI of the instance to delete (format: - 'projects//locations//instances/) - :param discovery_doc: Discovery document for building the Financial Services API - as described `here `__ + :param project_id: Required. The ID of the Google Cloud project that the service belongs to. + :param region: Required. The ID of the Google Cloud region that the service belongs to. + :param instance_id: Required. The ID of the instance, which is the final component of the + instances's name. :param gcp_conn_id: Identifier of connection to Google Cloud Platform. Defaults to "google_cloud_default". + :param timeout: The maximum time to wait for a long-running operation to complete. """ # [START howto_operator_financial_services_get_instance_template_fields] @@ -154,10 +154,10 @@ class FinancialServicesGetInstanceOperator(GoogleCloudBaseOperator): For more information on how to use this operator, take a look at the guide: :ref:`howto/operator:FinancialServicesGetInstanceOperator` - :param instance_resource_uri: URI of the instance to get (format: - 'projects//locations//instances/) - :param discovery_doc: Discovery document for building the Financial Services API - as described `here `__ + :param project_id: Required. The ID of the Google Cloud project that the service belongs to. + :param region: Required. The ID of the Google Cloud region that the service belongs to. + :param instance_id: Required. The ID of the instance, which is the final component of the + instances's name. :param gcp_conn_id: Identifier of connection to Google Cloud Platform. Defaults to "google_cloud_default". """ diff --git a/providers/src/airflow/providers/google/provider.yaml b/providers/src/airflow/providers/google/provider.yaml index ed7326796365..e3dc4d01f2fb 100644 --- a/providers/src/airflow/providers/google/provider.yaml +++ b/providers/src/airflow/providers/google/provider.yaml @@ -751,9 +751,6 @@ sensors: - integration-name: Google Dataproc Metastore python-modules: - airflow.providers.google.cloud.sensors.dataproc_metastore - - integration-name: Google Cloud Financial Services - python-modules: - - airflow.providers.google.cloud.sensors.financial_services - integration-name: Google Cloud Storage (GCS) python-modules: - airflow.providers.google.cloud.sensors.gcs