diff --git a/airflow/contrib/example_dags/example_gcp_dlp_operator.py b/airflow/contrib/example_dags/example_gcp_dlp_operator.py new file mode 100644 index 0000000000000..0786db703223a --- /dev/null +++ b/airflow/contrib/example_dags/example_gcp_dlp_operator.py @@ -0,0 +1,82 @@ +# -*- coding: utf-8 -*- +# +# 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. + +""" +Example Airflow DAG that execute the following tasks using +Cloud DLP service in the Google Cloud Platform: +1) Creating a content inspect template; +2) Using the created template to inspect content; +3) Deleting the template from GCP . +""" + +import os +from google.cloud.dlp_v2.types import ContentItem, InspectConfig, InspectTemplate + +import airflow +from airflow.models import DAG +from airflow.contrib.operators.gcp_dlp_operator import ( + CloudDLPCreateInspectTemplateOperator, + CloudDLPDeleteInspectTemplateOperator, + CloudDLPInspectContentOperator, +) + + +default_args = {"start_date": airflow.utils.dates.days_ago(1)} + + +GCP_PROJECT = os.environ.get("GCP_PROJECT_ID", "example-project") +TEMPLATE_ID = "dlp-inspect-8034856" +ITEM = ContentItem( + table={ + "headers": [{"name": "column1"}], + "rows": [{"values": [{"string_value": "My phone number is (206) 555-0123"}]}], + } +) +INSPECT_CONFIG = InspectConfig( + info_types=[{"name": "PHONE_NUMBER"}, {"name": "US_TOLLFREE_PHONE_NUMBER"}] +) +INSPECT_TEMPLATE = InspectTemplate(inspect_config=INSPECT_CONFIG) + + +with DAG("example_gcp_dlp", default_args=default_args, schedule_interval=None) as dag: + create_template = CloudDLPCreateInspectTemplateOperator( + project_id=GCP_PROJECT, + inspect_template=INSPECT_TEMPLATE, + template_id=TEMPLATE_ID, + task_id="create_template", + xcom_push=True, + dag=dag, + ) + + inspect_content = CloudDLPInspectContentOperator( + task_id="inpsect_content", + project_id=GCP_PROJECT, + item=ITEM, + inspect_template_name="{{ task_instance.xcom_pull('create_template', key='return_value')['name'] }}", + dag=dag, + ) + + delete_template = CloudDLPDeleteInspectTemplateOperator( + task_id="delete_template", + template_id=TEMPLATE_ID, + project_id=GCP_PROJECT, + dag=dag, + ) + + create_template > inspect_content > delete_template diff --git a/airflow/contrib/hooks/gcp_dlp_hook.py b/airflow/contrib/hooks/gcp_dlp_hook.py new file mode 100644 index 0000000000000..8935d5e15cb35 --- /dev/null +++ b/airflow/contrib/hooks/gcp_dlp_hook.py @@ -0,0 +1,1779 @@ +# -*- coding: utf-8 -*- +# +# 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. + +""" +This module contains a CloudDLPHook +which allows you to connect to GCP Cloud DLP service. +""" + +import re +import time +from google.cloud.dlp_v2 import DlpServiceClient +from google.cloud.dlp_v2.types import DlpJob + +from airflow import AirflowException +from airflow.contrib.hooks.gcp_api_base_hook import GoogleCloudBaseHook + +DLP_JOB_PATH_PATTERN = '^projects/[^/]+/dlpJobs/(?P.*?)$' +# Time to sleep between active checks of the operation results +TIME_TO_SLEEP_IN_SECONDS = 1 + + +# pylint: disable=R0904, C0302 +class CloudDLPHook(GoogleCloudBaseHook): + """ + Hook for Google Cloud Data Loss Prevention (DLP) APIs. + Cloud DLP allows clients to detect the presence of Personally Identifiable + Information (PII) and other privacy-sensitive data in user-supplied, + unstructured data streams, like text blocks or images. The service also + includes methods for sensitive data redaction and scheduling of data scans + on Google Cloud Platform based data sets. + + :param gcp_conn_id: The connection ID to use when fetching connection info. + :type gcp_conn_id: str + :param delegate_to: The account to impersonate, if any. + For this to work, the service account making the request must have + domain-wide delegation enabled. + :type delegate_to: str + """ + + def __init__(self, + gcp_conn_id="google_cloud_default", + delegate_to=None): + super().__init__(gcp_conn_id, delegate_to) + self._client = None + + def get_conn(self): + """ + Provides a client for interacting with the Cloud DLP API. + + :return: GCP Cloud DLP API Client + :rtype: google.cloud.dlp_v2.DlpServiceClient + """ + if not self._client: + self._client = DlpServiceClient(credentials=self._get_credentials()) + return self._client + + @GoogleCloudBaseHook.catch_http_exception + @GoogleCloudBaseHook.fallback_to_default_project_id + def cancel_dlp_job( + self, dlp_job_id, project_id=None, retry=None, timeout=None, metadata=None + ): + """ + Starts asynchronous cancellation on a long-running DLP job. + + :param dlp_job_id: ID of the DLP job resource to be cancelled. + :type dlp_job_id: str + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. If set to None or missing, the default project_id + from the GCP connection is used. + :type project_id: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + """ + + client = self.get_conn() + + if not dlp_job_id: + raise AirflowException( + "Please provide the ID of the DLP job resource to be cancelled." + ) + + name = DlpServiceClient.dlp_job_path(project_id, dlp_job_id) + client.cancel_dlp_job( + name=name, retry=retry, timeout=timeout, metadata=metadata + ) + + @GoogleCloudBaseHook.catch_http_exception + def create_deidentify_template( + self, + organization_id=None, + project_id=None, + deidentify_template=None, + template_id=None, + retry=None, + timeout=None, + metadata=None, + ): + """ + Creates a deidentify template for re-using frequently used configuration for + de-identifying content, images, and storage. + + :param organization_id: (Optional) The organization ID. Required to set this + field if parent resource is an organzation. + :type organization_id: str + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. Only set this field if the parent resource is + a project instead of an organzation. + :type project_id: str + :param deidentify_template: (Optional) The deidentify template to create. + :type deidentify_template: dict or google.cloud.dlp_v2.types.DeidentifyTemplate + :param template_id: (Optional) The template ID. + :type template_id: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :rtype: google.cloud.dlp_v2.types.DeidentifyTemplate + """ + + client = self.get_conn() + # Handle project_id from connection configuration + project_id = project_id or self.project_id + + if organization_id: + parent = DlpServiceClient.organization_path(organization_id) + elif project_id: + parent = DlpServiceClient.project_path(project_id) + else: + raise AirflowException( + "Please provide either organization_id or project_id." + ) + + return client.create_deidentify_template( + parent=parent, + deidentify_template=deidentify_template, + template_id=template_id, + retry=retry, + timeout=timeout, + metadata=metadata, + ) + + @GoogleCloudBaseHook.catch_http_exception + @GoogleCloudBaseHook.fallback_to_default_project_id + def create_dlp_job( + self, + project_id=None, + inspect_job=None, + risk_job=None, + job_id=None, + retry=None, + timeout=None, + metadata=None, + wait_until_finished=True + ): + """ + Creates a new job to inspect storage or calculate risk metrics. + + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. If set to None or missing, the default + project_id from the GCP connection is used. + :type project_id: str + :param inspect_job: (Optional) The configuration for the inspect job. + :type inspect_job: dict or google.cloud.dlp_v2.types.InspectJobConfig + :param risk_job: (Optional) The configuration for the risk job. + :type risk_job: dict or google.cloud.dlp_v2.types.RiskAnalysisJobConfig + :param job_id: (Optional) The job ID. + :type job_id: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :param wait_until_finished: (Optional) If true, it will keep polling the job state + until it is set to DONE. + :type wait_until_finished: bool + :rtype: google.cloud.dlp_v2.types.DlpJob + """ + + client = self.get_conn() + + parent = DlpServiceClient.project_path(project_id) + job = client.create_dlp_job( + parent=parent, + inspect_job=inspect_job, + risk_job=risk_job, + job_id=job_id, + retry=retry, + timeout=timeout, + metadata=metadata, + ) + + if wait_until_finished: + pattern = re.compile(DLP_JOB_PATH_PATTERN, re.IGNORECASE) + match = pattern.match(job.name) + job_name = match.groupdict()['job'] + + while wait_until_finished: + job = self.get_dlp_job( + dlp_job_id=job_name, + project_id=project_id) + + self.log.info( + 'DLP job {} state: {}.'.format( + job.name, + DlpJob.JobState.Name(job.state) + ) + ) + + if job.state == DlpJob.JobState.DONE: + return job + elif job.state in [DlpJob.JobState.PENDING, + DlpJob.JobState.RUNNING, + DlpJob.JobState.JOB_STATE_UNSPECIFIED]: + time.sleep(TIME_TO_SLEEP_IN_SECONDS) + else: + raise AirflowException( + 'Stopped polling DLP job state. DLP job {} state: {}.' + .format( + job.name, + DlpJob.JobState.Name(job.state) + ) + ) + return job + + @GoogleCloudBaseHook.catch_http_exception + def create_inspect_template( + self, + organization_id=None, + project_id=None, + inspect_template=None, + template_id=None, + retry=None, + timeout=None, + metadata=None, + ): + """ + Creates an inspect template for re-using frequently used configuration for + inspecting content, images, and storage. + + :param organization_id: (Optional) The organization ID. Required to set this + field if parent resource is an organzation. + :type organization_id: str + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. Only set this field if the parent resource is + a project instead of an organzation. + :type project_id: str + :param inspect_template: (Optional) The inspect template to create. + :type inspect_template: dict or google.cloud.dlp_v2.types.InspectTemplate + :param template_id: (Optional) The template ID. + :type template_id: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :rtype: google.cloud.dlp_v2.types.InspectTemplate + """ + + client = self.get_conn() + + # Handle project_id from connection configuration + project_id = project_id or self.project_id + + if organization_id: + parent = DlpServiceClient.organization_path(organization_id) + elif project_id: + parent = DlpServiceClient.project_path(project_id) + else: + raise AirflowException( + "Please provide either organization_id or project_id." + ) + + return client.create_inspect_template( + parent=parent, + inspect_template=inspect_template, + template_id=template_id, + retry=retry, + timeout=timeout, + metadata=metadata, + ) + + @GoogleCloudBaseHook.catch_http_exception + @GoogleCloudBaseHook.fallback_to_default_project_id + def create_job_trigger( + self, + project_id=None, + job_trigger=None, + trigger_id=None, + retry=None, + timeout=None, + metadata=None, + ): + """ + Creates a job trigger to run DLP actions such as scanning storage for sensitive + information on a set schedule. + + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. If set to None or missing, the default + project_id from the GCP connection is used. + :type project_id: str + :param job_trigger: (Optional) The job trigger to create. + :type job_trigger: dict or google.cloud.dlp_v2.types.JobTrigger + :param trigger_id: (Optional) The job trigger ID. + :type trigger_id: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :rtype: google.cloud.dlp_v2.types.JobTrigger + """ + + client = self.get_conn() + + parent = DlpServiceClient.project_path(project_id) + return client.create_job_trigger( + parent=parent, + job_trigger=job_trigger, + trigger_id=trigger_id, + retry=retry, + timeout=timeout, + metadata=metadata, + ) + + @GoogleCloudBaseHook.catch_http_exception + def create_stored_info_type( + self, + organization_id=None, + project_id=None, + config=None, + stored_info_type_id=None, + retry=None, + timeout=None, + metadata=None, + ): + """ + Creates a pre-built stored info type to be used for inspection. + + :param organization_id: (Optional) The organization ID. Required to set this + field if parent resource is an organzation. + :type organization_id: str + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. Only set this field if the parent resource is + a project instead of an organzation. + :type project_id: str + :param config: (Optional) The config for the stored info type. + :type config: dict or google.cloud.dlp_v2.types.StoredInfoTypeConfig + :param stored_info_type_id: (Optional) The stored info type ID. + :type stored_info_type_id: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :rtype: google.cloud.dlp_v2.types.StoredInfoType + """ + + client = self.get_conn() + + # Handle project_id from connection configuration + project_id = project_id or self.project_id + + if organization_id: + parent = DlpServiceClient.organization_path(organization_id) + elif project_id: + parent = DlpServiceClient.project_path(project_id) + else: + raise AirflowException( + "Please provide either organization_id or project_id." + ) + + return client.create_stored_info_type( + parent=parent, + config=config, + stored_info_type_id=stored_info_type_id, + retry=retry, + timeout=timeout, + metadata=metadata, + ) + + @GoogleCloudBaseHook.catch_http_exception + @GoogleCloudBaseHook.fallback_to_default_project_id + def deidentify_content( + self, + project_id=None, + deidentify_config=None, + inspect_config=None, + item=None, + inspect_template_name=None, + deidentify_template_name=None, + retry=None, + timeout=None, + metadata=None, + ): + """ + De-identifies potentially sensitive info from a content item. This method has limits + on input size and output size. + + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. If set to None or missing, the default + project_id from the GCP connection is used. + :type project_id: str + :param deidentify_config: (Optional) Configuration for the de-identification of the + content item. Items specified here will override the template referenced by the + deidentify_template_name argument. + :type deidentify_config: dict or google.cloud.dlp_v2.types.DeidentifyConfig + :param inspect_config: (Optional) Configuration for the inspector. Items specified + here will override the template referenced by the inspect_template_name argument. + :type inspect_config: dict or google.cloud.dlp_v2.types.InspectConfig + :param item: (Optional) The item to de-identify. Will be treated as text. + :type item: dict or google.cloud.dlp_v2.types.ContentItem + :param inspect_template_name: (Optional) Optional template to use. Any configuration + directly specified in inspect_config will override those set in the template. + :type inspect_template_name: str + :param deidentify_template_name: (Optional) Optional template to use. Any + configuration directly specified in deidentify_config will override those set + in the template. + :type deidentify_template_name: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :rtype: google.cloud.dlp_v2.types.DeidentifyContentResponse + """ + + client = self.get_conn() + + parent = DlpServiceClient.project_path(project_id) + return client.deidentify_content( + parent=parent, + deidentify_config=deidentify_config, + inspect_config=inspect_config, + item=item, + inspect_template_name=inspect_template_name, + deidentify_template_name=deidentify_template_name, + retry=retry, + timeout=timeout, + metadata=metadata, + ) + + @GoogleCloudBaseHook.catch_http_exception + def delete_deidentify_template( + self, + template_id, + organization_id=None, + project_id=None, + retry=None, + timeout=None, + metadata=None + ): + """ + Deletes a deidentify template. + + :param template_id: The ID of deidentify template to be deleted. + :type template_id: str + :param organization_id: (Optional) The organization ID. Required to set this + field if parent resource is an organzation. + :type organization_id: str + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. Only set this field if the parent resource is + a project instead of an organzation. + :type project_id: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + """ + + client = self.get_conn() + + if not template_id: + raise AirflowException( + "Please provide the ID of deidentify template to be deleted." + ) + + # Handle project_id from connection configuration + project_id = project_id or self.project_id + + if organization_id: + name = DlpServiceClient.organization_deidentify_template_path(organization_id, template_id) + elif project_id: + name = DlpServiceClient.project_deidentify_template_path(project_id, template_id) + else: + raise AirflowException( + "Please provide either organization_id or project_id." + ) + + client.delete_deidentify_template( + name=name, retry=retry, timeout=timeout, metadata=metadata + ) + + @GoogleCloudBaseHook.catch_http_exception + @GoogleCloudBaseHook.fallback_to_default_project_id + def delete_dlp_job( + self, dlp_job_id, project_id=None, retry=None, timeout=None, metadata=None + ): + """ + Deletes a long-running DLP job. This method indicates that the client is no longer + interested in the DLP job result. The job will be cancelled if possible. + + :param dlp_job_id: The ID of the DLP job resource to be cancelled. + :type dlp_job_id: str + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. If set to None or missing, the default + project_id from the GCP connection is used. + :type project_id: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + """ + + client = self.get_conn() + + if not dlp_job_id: + raise AirflowException( + "Please provide the ID of the DLP job resource to be cancelled." + ) + + name = DlpServiceClient.dlp_job_path(project_id, dlp_job_id) + client.delete_dlp_job( + name=name, retry=retry, timeout=timeout, metadata=metadata + ) + + @GoogleCloudBaseHook.catch_http_exception + def delete_inspect_template( + self, + template_id, + organization_id=None, + project_id=None, + retry=None, + timeout=None, + metadata=None + ): + """ + Deletes an inspect template. + + :param template_id: The ID of the inspect template to be deleted. + :type template_id: str + :param organization_id: (Optional) The organization ID. Required to set this + field if parent resource is an organzation. + :type organization_id: str + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. Only set this field if the parent resource is + a project instead of an organzation. + :type project_id: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + """ + + client = self.get_conn() + + if not template_id: + raise AirflowException( + "Please provide the ID of the inspect template to be deleted." + ) + + # Handle project_id from connection configuration + project_id = project_id or self.project_id + + if organization_id: + name = DlpServiceClient.organization_inspect_template_path(organization_id, template_id) + elif project_id: + name = DlpServiceClient.project_inspect_template_path(project_id, template_id) + else: + raise AirflowException( + "Please provide either organization_id or project_id." + ) + + client.delete_inspect_template( + name=name, retry=retry, timeout=timeout, metadata=metadata + ) + + @GoogleCloudBaseHook.catch_http_exception + @GoogleCloudBaseHook.fallback_to_default_project_id + def delete_job_trigger( + self, + job_trigger_id, + project_id=None, + retry=None, + timeout=None, + metadata=None + ): + """ + Deletes a job trigger. + + :param job_trigger_id: The ID of the DLP job trigger to be deleted. + :type job_trigger_id: str + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. If set to None or missing, the default + project_id from the GCP connection is used. + :type project_id: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + """ + + client = self.get_conn() + + if not job_trigger_id: + raise AirflowException( + "Please provide the ID of the DLP job trigger to be deleted." + ) + + name = DlpServiceClient.project_job_trigger_path(project_id, job_trigger_id) + client.delete_job_trigger( + name=name, retry=retry, timeout=timeout, metadata=metadata + ) + + @GoogleCloudBaseHook.catch_http_exception + def delete_stored_info_type( + self, + stored_info_type_id, + organization_id=None, + project_id=None, + retry=None, + timeout=None, + metadata=None + ): + """ + Deletes a stored info type. + + :param stored_info_type_id: The ID of the stored info type to be deleted. + :type stored_info_type_id: str + :param organization_id: (Optional) The organization ID. Required to set this + field if parent resource is an organzation. + :type organization_id: str + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. Only set this field if the parent resource is + a project instead of an organzation. + :type project_id: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + """ + + client = self.get_conn() + + if not stored_info_type_id: + raise AirflowException( + "Please provide the ID of the stored info type to be deleted." + ) + + # Handle project_id from connection configuration + project_id = project_id or self.project_id + + if organization_id: + name = DlpServiceClient.organization_stored_info_type_path( + organization_id, stored_info_type_id + ) + elif project_id: + name = DlpServiceClient.project_stored_info_type_path( + project_id, + stored_info_type_id + ) + else: + raise AirflowException( + "Please provide either organization_id or project_id." + ) + + client.delete_stored_info_type( + name=name, retry=retry, timeout=timeout, metadata=metadata + ) + + @GoogleCloudBaseHook.catch_http_exception + def get_deidentify_template( + self, + template_id, + organization_id=None, + project_id=None, + retry=None, + timeout=None, + metadata=None + ): + """ + Gets a deidentify template. + + :param template_id: The ID of deidentify template to be read. + :type template_id: str + :param organization_id: (Optional) The organization ID. Required to set this + field if parent resource is an organzation. + :type organization_id: str + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. Only set this field if the parent resource is + a project instead of an organzation. + :type project_id: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :rtype: google.cloud.dlp_v2.types.DeidentifyTemplate + """ + + client = self.get_conn() + + if not template_id: + raise AirflowException( + "Please provide the ID of the deidentify template to be read." + ) + + # Handle project_id from connection configuration + project_id = project_id or self.project_id + + if organization_id: + name = DlpServiceClient.organization_deidentify_template_path(organization_id, template_id) + elif project_id: + name = DlpServiceClient.project_deidentify_template_path(project_id, template_id) + else: + raise AirflowException( + "Please provide either organization_id or project_id." + ) + + return client.get_deidentify_template( + name=name, retry=retry, timeout=timeout, metadata=metadata + ) + + @GoogleCloudBaseHook.catch_http_exception + @GoogleCloudBaseHook.fallback_to_default_project_id + def get_dlp_job( + self, + dlp_job_id, + project_id=None, + retry=None, + timeout=None, + metadata=None + ): + """ + Gets the latest state of a long-running Dlp Job. + + :param dlp_job_id: The ID of the DLP job resource to be read. + :type dlp_job_id: str + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. If set to None or missing, the default + project_id from the GCP connection is used. + :type project_id: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :rtype: google.cloud.dlp_v2.types.DlpJob + """ + + client = self.get_conn() + + if not dlp_job_id: + raise AirflowException( + "Please provide the ID of the DLP job resource to be read." + ) + + name = DlpServiceClient.dlp_job_path(project_id, dlp_job_id) + return client.get_dlp_job( + name=name, retry=retry, timeout=timeout, metadata=metadata + ) + + @GoogleCloudBaseHook.catch_http_exception + def get_inspect_template( + self, + template_id, + organization_id=None, + project_id=None, + retry=None, + timeout=None, + metadata=None + ): + """ + Gets an inspect template. + + :param template_id: The ID of inspect template to be read. + :type template_id: str + :param organization_id: (Optional) The organization ID. Required to set this + field if parent resource is an organzation. + :type organization_id: str + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. Only set this field if the parent resource is + a project instead of an organzation. + :type project_id: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :rtype: google.cloud.dlp_v2.types.InspectTemplate + """ + + client = self.get_conn() + + if not template_id: + raise AirflowException( + "Please provide the ID of the inspect template to be read." + ) + + # Handle project_id from connection configuration + project_id = project_id or self.project_id + + if organization_id: + name = DlpServiceClient.organization_inspect_template_path(organization_id, template_id) + elif project_id: + name = DlpServiceClient.project_inspect_template_path(project_id, template_id) + else: + raise AirflowException( + "Please provide either organization_id or project_id." + ) + + return client.get_inspect_template( + name=name, retry=retry, timeout=timeout, metadata=metadata + ) + + @GoogleCloudBaseHook.catch_http_exception + @GoogleCloudBaseHook.fallback_to_default_project_id + def get_job_trigger( + self, + job_trigger_id, + project_id=None, + retry=None, + timeout=None, + metadata=None + ): + """ + Gets a DLP job trigger. + + :param job_trigger_id: The ID of the DLP job trigger to be read. + :type job_trigger_id: str + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. If set to None or missing, the default + project_id from the GCP connection is used. + :type project_id: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :rtype: google.cloud.dlp_v2.types.JobTrigger + """ + + client = self.get_conn() + + if not job_trigger_id: + raise AirflowException( + "Please provide the ID of the DLP job trigger to be read." + ) + + name = DlpServiceClient.project_job_trigger_path(project_id, job_trigger_id) + return client.get_job_trigger( + name=name, retry=retry, timeout=timeout, metadata=metadata + ) + + @GoogleCloudBaseHook.catch_http_exception + def get_stored_info_type( + self, + stored_info_type_id, + organization_id=None, + project_id=None, + retry=None, + timeout=None, + metadata=None + ): + """ + Gets a stored info type. + + :param stored_info_type_id: The ID of the stored info type to be read. + :type stored_info_type_id: str + :param organization_id: (Optional) The organization ID. Required to set this + field if parent resource is an organzation. + :type organization_id: str + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. Only set this field if the parent resource is + a project instead of an organzation. + :type project_id: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :rtype: google.cloud.dlp_v2.types.StoredInfoType + """ + + client = self.get_conn() + + if not stored_info_type_id: + raise AirflowException( + "Please provide the ID of the stored info type to be read." + ) + + # Handle project_id from connection configuration + project_id = project_id or self.project_id + + if organization_id: + name = DlpServiceClient.organization_stored_info_type_path( + organization_id, stored_info_type_id + ) + elif project_id: + name = DlpServiceClient.project_stored_info_type_path( + project_id, + stored_info_type_id + ) + else: + raise AirflowException( + "Please provide either organization_id or project_id." + ) + + return client.get_stored_info_type( + name=name, retry=retry, timeout=timeout, metadata=metadata + ) + + @GoogleCloudBaseHook.catch_http_exception + @GoogleCloudBaseHook.fallback_to_default_project_id + def inspect_content( + self, + project_id=None, + inspect_config=None, + item=None, + inspect_template_name=None, + retry=None, + timeout=None, + metadata=None, + ): + """ + Finds potentially sensitive info in content. This method has limits on input size, + processing time, and output size. + + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. If set to None or missing, the default + project_id from the GCP connection is used. + :type project_id: str + :param inspect_config: (Optional) Configuration for the inspector. Items specified + here will override the template referenced by the inspect_template_name argument. + :type inspect_config: dict or google.cloud.dlp_v2.types.InspectConfig + :param item: (Optional) The item to de-identify. Will be treated as text. + :type item: dict or google.cloud.dlp_v2.types.ContentItem + :param inspect_template_name: (Optional) Optional template to use. Any configuration + directly specified in inspect_config will override those set in the template. + :type inspect_template_name: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :rtype: google.cloud.dlp_v2.types.InspectContentResponse + """ + + client = self.get_conn() + + parent = DlpServiceClient.project_path(project_id) + return client.inspect_content( + parent=parent, + inspect_config=inspect_config, + item=item, + inspect_template_name=inspect_template_name, + retry=retry, + timeout=timeout, + metadata=metadata, + ) + + @GoogleCloudBaseHook.catch_http_exception + def list_deidentify_templates( + self, + organization_id=None, + project_id=None, + page_size=None, + order_by=None, + retry=None, + timeout=None, + metadata=None, + ): + """ + Lists deidentify templates. + + :param organization_id: (Optional) The organization ID. Required to set this + field if parent resource is an organzation. + :type organization_id: str + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. Only set this field if the parent resource is + a project instead of an organzation. + :type project_id: str + :param page_size: (Optional) The maximum number of resources contained in the + underlying API response. + :type page_size: int + :param order_by: (Optional) Optional comma separated list of fields to order by, + followed by asc or desc postfix. + :type order_by: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :rtype: list[google.cloud.dlp_v2.types.DeidentifyTemplate] + """ + + client = self.get_conn() + + # Handle project_id from connection configuration + project_id = project_id or self.project_id + + if organization_id: + parent = DlpServiceClient.organization_path(organization_id) + elif project_id: + parent = DlpServiceClient.project_path(project_id) + else: + raise AirflowException( + "Please provide either organization_id or project_id." + ) + + results = client.list_deidentify_templates( + parent=parent, + page_size=page_size, + order_by=order_by, + retry=retry, + timeout=timeout, + metadata=metadata, + ) + + return list(results) + + @GoogleCloudBaseHook.catch_http_exception + @GoogleCloudBaseHook.fallback_to_default_project_id + def list_dlp_jobs( + self, + project_id=None, + results_filter=None, + page_size=None, + job_type=None, + order_by=None, + retry=None, + timeout=None, + metadata=None, + ): + """ + Lists DLP jobs that match the specified filter in the request. + + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. If set to None or missing, the default + project_id from the GCP connection is used. + :type project_id: str + :param results_filter: (Optional) Filter used to specify a subset of results. + :type results_filter: str + :param page_size: (Optional) The maximum number of resources contained in the + underlying API response. + :type page_size: int + :param job_type: (Optional) The type of job. + :type job_type: str + :param order_by: (Optional) Optional comma separated list of fields to order by, + followed by asc or desc postfix. + :type order_by: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :rtype: list[google.cloud.dlp_v2.types.DlpJob] + """ + + client = self.get_conn() + + parent = DlpServiceClient.project_path(project_id) + results = client.list_dlp_jobs( + parent=parent, + filter_=results_filter, + page_size=page_size, + type_=job_type, + order_by=order_by, + retry=retry, + timeout=timeout, + metadata=metadata, + ) + return list(results) + + @GoogleCloudBaseHook.catch_http_exception + def list_info_types( + self, + language_code=None, + results_filter=None, + retry=None, + timeout=None, + metadata=None, + ): + """ + Returns a list of the sensitive information types that the DLP API supports. + + :param language_code: (Optional) Optional BCP-47 language code for localized info + type friendly names. If omitted, or if localized strings are not available, + en-US strings will be returned. + :type language_code: str + :param results_filter: (Optional) Filter used to specify a subset of results. + :type results_filter: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :rtype: google.cloud.dlp_v2.types.ListInfoTypesResponse + """ + + client = self.get_conn() + + return client.list_info_types( + language_code=language_code, + filter_=results_filter, + retry=retry, + timeout=timeout, + metadata=metadata, + ) + + @GoogleCloudBaseHook.catch_http_exception + def list_inspect_templates( + self, + organization_id=None, + project_id=None, + page_size=None, + order_by=None, + retry=None, + timeout=None, + metadata=None, + ): + """ + Lists inspect templates. + + :param organization_id: (Optional) The organization ID. Required to set this + field if parent resource is an organzation. + :type organization_id: str + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. Only set this field if the parent resource is + a project instead of an organzation. + :type project_id: str + :param page_size: (Optional) The maximum number of resources contained in the + underlying API response. + :type page_size: int + :param order_by: (Optional) Optional comma separated list of fields to order by, + followed by asc or desc postfix. + :type order_by: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :rtype: list[google.cloud.dlp_v2.types.InspectTemplate] + """ + + client = self.get_conn() + + # Handle project_id from connection configuration + project_id = project_id or self.project_id + + if organization_id: + parent = DlpServiceClient.organization_path(organization_id) + elif project_id: + parent = DlpServiceClient.project_path(project_id) + else: + raise AirflowException( + "Please provide either organization_id or project_id." + ) + + results = client.list_inspect_templates( + parent=parent, + page_size=page_size, + order_by=order_by, + retry=retry, + timeout=timeout, + metadata=metadata, + ) + return list(results) + + @GoogleCloudBaseHook.catch_http_exception + @GoogleCloudBaseHook.fallback_to_default_project_id + def list_job_triggers( + self, + project_id=None, + page_size=None, + order_by=None, + results_filter=None, + retry=None, + timeout=None, + metadata=None, + ): + """ + Lists job triggers. + + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. If set to None or missing, the default + project_id from the GCP connection is used. + :type project_id: str + :param page_size: (Optional) The maximum number of resources contained in the + underlying API response. + :type page_size: int + :param order_by: (Optional) Optional comma separated list of fields to order by, + followed by asc or desc postfix. + :type order_by: str + :param results_filter: (Optional) Filter used to specify a subset of results. + :type results_filter: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :rtype: list[google.cloud.dlp_v2.types.JobTrigger] + """ + + client = self.get_conn() + + parent = DlpServiceClient.project_path(project_id) + results = client.list_job_triggers( + parent=parent, + page_size=page_size, + order_by=order_by, + filter_=results_filter, + retry=retry, + timeout=timeout, + metadata=metadata, + ) + return list(results) + + @GoogleCloudBaseHook.catch_http_exception + def list_stored_info_types( + self, + organization_id=None, + project_id=None, + page_size=None, + order_by=None, + retry=None, + timeout=None, + metadata=None, + ): + """ + Lists stored info types. + + :param organization_id: (Optional) The organization ID. Required to set this + field if parent resource is an organzation. + :type organization_id: str + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. Only set this field if the parent resource is + a project instead of an organzation. + :type project_id: str + :param page_size: (Optional) The maximum number of resources contained in the + underlying API response. + :type page_size: int + :param order_by: (Optional) Optional comma separated list of fields to order by, + followed by asc or desc postfix. + :type order_by: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :rtype: list[google.cloud.dlp_v2.types.StoredInfoType] + """ + + client = self.get_conn() + + # Handle project_id from connection configuration + project_id = project_id or self.project_id + + if organization_id: + parent = DlpServiceClient.organization_path(organization_id) + elif project_id: + parent = DlpServiceClient.project_path(project_id) + else: + raise AirflowException( + "Please provide either organization_id or project_id." + ) + + results = client.list_stored_info_types( + parent=parent, + page_size=page_size, + order_by=order_by, + retry=retry, + timeout=timeout, + metadata=metadata, + ) + return list(results) + + @GoogleCloudBaseHook.catch_http_exception + @GoogleCloudBaseHook.fallback_to_default_project_id + def redact_image( + self, + project_id=None, + inspect_config=None, + image_redaction_configs=None, + include_findings=None, + byte_item=None, + retry=None, + timeout=None, + metadata=None, + ): + """ + Redacts potentially sensitive info from an image. This method has limits on + input size, processing time, and output size. + + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. If set to None or missing, the default + project_id from the GCP connection is used. + :type project_id: str + :param inspect_config: (Optional) Configuration for the inspector. Items specified + here will override the template referenced by the inspect_template_name argument. + :type inspect_config: dict or google.cloud.dlp_v2.types.InspectConfig + :param image_redaction_configs: (Optional) The configuration for specifying what + content to redact from images. + :type image_redaction_configs: list[dict] or list[google.cloud.dlp_v2.types.ImageRedactionConfig] + :param include_findings: (Optional) Whether the response should include findings + along with the redacted image. + :type include_findings: bool + :param byte_item: (Optional) The content must be PNG, JPEG, SVG or BMP. + :type byte_item: dict or google.cloud.dlp_v2.types.ByteContentItem + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :rtype: google.cloud.dlp_v2.types.RedactImageResponse + """ + + client = self.get_conn() + + parent = DlpServiceClient.project_path(project_id) + return client.redact_image( + parent=parent, + inspect_config=inspect_config, + image_redaction_configs=image_redaction_configs, + include_findings=include_findings, + byte_item=byte_item, + retry=retry, + timeout=timeout, + metadata=metadata, + ) + + @GoogleCloudBaseHook.catch_http_exception + @GoogleCloudBaseHook.fallback_to_default_project_id + def reidentify_content( + self, + project_id=None, + reidentify_config=None, + inspect_config=None, + item=None, + inspect_template_name=None, + reidentify_template_name=None, + retry=None, + timeout=None, + metadata=None, + ): + """ + Re-identifies content that has been de-identified. + + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. If set to None or missing, the default + project_id from the GCP connection is used. + :type project_id: str + :param reidentify_config: (Optional) Configuration for the re-identification of + the content item. + :type reidentify_config: dict or google.cloud.dlp_v2.types.DeidentifyConfig + :param inspect_config: (Optional) Configuration for the inspector. + :type inspect_config: dict or google.cloud.dlp_v2.types.InspectConfig + :param item: (Optional) The item to re-identify. Will be treated as text. + :type item: dict or google.cloud.dlp_v2.types.ContentItem + :param inspect_template_name: (Optional) Optional template to use. Any configuration + directly specified in inspect_config will override those set in the template. + :type inspect_template_name: str + :param reidentify_template_name: (Optional) Optional template to use. References an + instance of deidentify template. Any configuration directly specified in + reidentify_config or inspect_config will override those set in the template. + :type reidentify_template_name: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :rtype: google.cloud.dlp_v2.types.ReidentifyContentResponse + """ + + client = self.get_conn() + + parent = DlpServiceClient.project_path(project_id) + return client.reidentify_content( + parent=parent, + reidentify_config=reidentify_config, + inspect_config=inspect_config, + item=item, + inspect_template_name=inspect_template_name, + reidentify_template_name=reidentify_template_name, + retry=retry, + timeout=timeout, + metadata=metadata, + ) + + @GoogleCloudBaseHook.catch_http_exception + def update_deidentify_template( + self, + template_id, + organization_id=None, + project_id=None, + deidentify_template=None, + update_mask=None, + retry=None, + timeout=None, + metadata=None, + ): + """ + Updates the deidentify template. + + :param template_id: The ID of deidentify template to be updated. + :type template_id: str + :param organization_id: (Optional) The organization ID. Required to set this + field if parent resource is an organzation. + :type organization_id: str + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. Only set this field if the parent resource is + a project instead of an organzation. + :type project_id: str + :param deidentify_template: New deidentify template value. + :type deidentify_template: dict or google.cloud.dlp_v2.types.DeidentifyTemplate + :param update_mask: Mask to control which fields get updated. + :type update_mask: dict or google.cloud.dlp_v2.types.FieldMask + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :rtype: google.cloud.dlp_v2.types.DeidentifyTemplate + """ + + client = self.get_conn() + + if not template_id: + raise AirflowException( + "Please provide the ID of deidentify template to be updated." + ) + + # Handle project_id from connection configuration + project_id = project_id or self.project_id + + if organization_id: + name = DlpServiceClient.organization_deidentify_template_path(organization_id, template_id) + elif project_id: + name = DlpServiceClient.project_deidentify_template_path(project_id, template_id) + else: + raise AirflowException( + "Please provide either organization_id or project_id." + ) + + return client.update_deidentify_template( + name=name, + deidentify_template=deidentify_template, + update_mask=update_mask, + retry=retry, + timeout=timeout, + metadata=metadata, + ) + + @GoogleCloudBaseHook.catch_http_exception + def update_inspect_template( + self, + template_id, + organization_id=None, + project_id=None, + inspect_template=None, + update_mask=None, + retry=None, + timeout=None, + metadata=None, + ): + """ + Updates the inspect template. + + :param template_id: The ID of the inspect template to be updated. + :type template_id: str + :param organization_id: (Optional) The organization ID. Required to set this + field if parent resource is an organzation. + :type organization_id: str + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. Only set this field if the parent resource is + a project instead of an organzation. + :type project_id: str + :param inspect_template: New inspect template value. + :type inspect_template: dict or google.cloud.dlp_v2.types.InspectTemplate + :param update_mask: Mask to control which fields get updated. + :type update_mask: dict or google.cloud.dlp_v2.types.FieldMask + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :rtype: google.cloud.dlp_v2.types.InspectTemplate + """ + + client = self.get_conn() + + if not template_id: + raise AirflowException( + "Please provide the ID of the inspect template to be updated." + ) + # Handle project_id from connection configuration + project_id = project_id or self.project_id + + if organization_id: + name = DlpServiceClient.organization_inspect_template_path(organization_id, template_id) + elif project_id: + name = DlpServiceClient.project_inspect_template_path(project_id, template_id) + else: + raise AirflowException( + "Please provide either organization_id or project_id." + ) + + return client.update_inspect_template( + name=name, + inspect_template=inspect_template, + update_mask=update_mask, + retry=retry, + timeout=timeout, + metadata=metadata, + ) + + @GoogleCloudBaseHook.catch_http_exception + @GoogleCloudBaseHook.fallback_to_default_project_id + def update_job_trigger( + self, + job_trigger_id, + project_id=None, + job_trigger=None, + update_mask=None, + retry=None, + timeout=None, + metadata=None, + ): + """ + Updates a job trigger. + + :param job_trigger_id: The ID of the DLP job trigger to be updated. + :type job_trigger_id: str + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. If set to None or missing, the default + project_id from the GCP connection is used. + :type project_id: str + :param job_trigger: New job trigger value. + :type job_trigger: dict or google.cloud.dlp_v2.types.JobTrigger + :param update_mask: Mask to control which fields get updated. + :type update_mask: dict or google.cloud.dlp_v2.types.FieldMask + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :rtype: google.cloud.dlp_v2.types.JobTrigger + """ + + client = self.get_conn() + + if not job_trigger_id: + raise AirflowException( + "Please provide the ID of the DLP job trigger to be updated." + ) + + name = DlpServiceClient.project_job_trigger_path(project_id, job_trigger_id) + return client.update_job_trigger( + name=name, + job_trigger=job_trigger, + update_mask=update_mask, + retry=retry, + timeout=timeout, + metadata=metadata, + ) + + @GoogleCloudBaseHook.catch_http_exception + def update_stored_info_type( + self, + stored_info_type_id, + organization_id=None, + project_id=None, + config=None, + update_mask=None, + retry=None, + timeout=None, + metadata=None, + ): + """ + Updates the stored info type by creating a new version. + + :param stored_info_type_id: The ID of the stored info type to be updated. + :type stored_info_type_id: str + :param organization_id: (Optional) The organization ID. Required to set this + field if parent resource is an organzation. + :type organization_id: str + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. Only set this field if the parent resource is + a project instead of an organzation. + :type project_id: str + :param config: Updated configuration for the stored info type. If not provided, a new + version of the stored info type will be created with the existing configuration. + :type config: dict or google.cloud.dlp_v2.types.StoredInfoTypeConfig + :param update_mask: Mask to control which fields get updated. + :type update_mask: dict or google.cloud.dlp_v2.types.FieldMask + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :rtype: google.cloud.dlp_v2.types.StoredInfoType + """ + + client = self.get_conn() + + if not stored_info_type_id: + raise AirflowException( + "Please provide the ID of the stored info type to be updated." + ) + + # Handle project_id from connection configuration + project_id = project_id or self.project_id + + if organization_id: + name = DlpServiceClient.organization_stored_info_type_path( + organization_id, stored_info_type_id + ) + elif project_id: + name = DlpServiceClient.project_stored_info_type_path( + project_id, + stored_info_type_id + ) + else: + raise AirflowException( + "Please provide either organization_id or project_id." + ) + + return client.update_stored_info_type( + name=name, + config=config, + update_mask=update_mask, + retry=retry, + timeout=timeout, + metadata=metadata, + ) diff --git a/airflow/contrib/operators/dataproc_operator.py b/airflow/contrib/operators/dataproc_operator.py index 288f4ef59cdf6..feaddbeb3a105 100644 --- a/airflow/contrib/operators/dataproc_operator.py +++ b/airflow/contrib/operators/dataproc_operator.py @@ -20,7 +20,7 @@ """ This module contains Google Dataproc operators. """ -# pylint: disable=too-many-lines +# pylint: disable=C0302 import ntpath import os diff --git a/airflow/contrib/operators/gcp_dlp_operator.py b/airflow/contrib/operators/gcp_dlp_operator.py new file mode 100644 index 0000000000000..05dba5d373e42 --- /dev/null +++ b/airflow/contrib/operators/gcp_dlp_operator.py @@ -0,0 +1,2121 @@ +# -*- coding: utf-8 -*- +# +# 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. + +# pylint: disable=R0913, C0302 +""" +This module contains various GCP Cloud DLP operators +which allow you to perform basic operations using +Cloud DLP. +""" + +from airflow.contrib.hooks.gcp_dlp_hook import CloudDLPHook +from airflow.models import BaseOperator +from airflow.utils.decorators import apply_defaults + + +class CloudDLPCancelDLPJobOperator(BaseOperator): + """ + Starts asynchronous cancellation on a long-running DlpJob. + + :param dlp_job_id: ID of the DLP job resource to be cancelled. + :type dlp_job_id: str + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. If set to None or missing, the default project_id + from the GCP connection is used. + :type project_id: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform. + :type gcp_conn_id: str + """ + + template_fields = ("dlp_job_id", "project_id", "gcp_conn_id") + + @apply_defaults + def __init__( + self, + dlp_job_id, + project_id=None, + retry=None, + timeout=None, + metadata=None, + gcp_conn_id="google_cloud_default", + *args, + **kwargs + ): + super().__init__(*args, **kwargs) + self.dlp_job_id = dlp_job_id + self.project_id = project_id + self.retry = retry + self.timeout = timeout + self.metadata = metadata + self.gcp_conn_id = gcp_conn_id + + def execute(self, context): + hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id) + hook.cancel_dlp_job( + dlp_job_id=self.dlp_job_id, + project_id=self.project_id, + retry=self.retry, + timeout=self.timeout, + metadata=self.metadata, + ) + + +class CloudDLPCreateDeidentifyTemplateOperator(BaseOperator): + """ + Creates a DeidentifyTemplate for re-using frequently used configuration for + de-identifying content, images, and storage. + + :param organization_id: (Optional) The organization ID. Required to set this + field if parent resource is an organzation. + :type organization_id: str + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. Only set this field if the parent resource is + a project instead of an organzation. + :type project_id: str + :param deidentify_template: (Optional) The DeidentifyTemplate to create. + :type deidentify_template: dict or google.cloud.dlp_v2.types.DeidentifyTemplate + :param template_id: (Optional) The template ID. + :type template_id: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform. + :type gcp_conn_id: str + :rtype: google.cloud.dlp_v2.types.DeidentifyTemplate + """ + + template_fields = ( + "organization_id", + "project_id", + "deidentify_template", + "template_id", + "gcp_conn_id", + ) + + @apply_defaults + def __init__( + self, + organization_id=None, + project_id=None, + deidentify_template=None, + template_id=None, + retry=None, + timeout=None, + metadata=None, + gcp_conn_id="google_cloud_default", + *args, + **kwargs + ): + super().__init__(*args, **kwargs) + self.organization_id = organization_id + self.project_id = project_id + self.deidentify_template = deidentify_template + self.template_id = template_id + self.retry = retry + self.timeout = timeout + self.metadata = metadata + self.gcp_conn_id = gcp_conn_id + + def execute(self, context): + hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id) + return hook.create_deidentify_template( + organization_id=self.organization_id, + project_id=self.project_id, + deidentify_template=self.deidentify_template, + template_id=self.template_id, + retry=self.retry, + timeout=self.timeout, + metadata=self.metadata, + ) + + +class CloudDLPCreateDLPJobOperator(BaseOperator): + """ + Creates a new job to inspect storage or calculate risk metrics. + + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. If set to None or missing, the default + project_id from the GCP connection is used. + :type project_id: str + :param inspect_job: (Optional) The configuration for the inspect job. + :type inspect_job: dict or google.cloud.dlp_v2.types.InspectJobConfig + :param risk_job: (Optional) The configuration for the risk job. + :type risk_job: dict or google.cloud.dlp_v2.types.RiskAnalysisJobConfig + :param job_id: (Optional) The job ID. + :type job_id: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :param wait_until_finished: (Optional) If true, it will keep polling the job state + until it is set to DONE. + :type wait_until_finished: bool + :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform. + :type gcp_conn_id: str + :rtype: google.cloud.dlp_v2.types.DlpJob + """ + + template_fields = ("project_id", "inspect_job", "risk_job", "job_id", "gcp_conn_id") + + @apply_defaults + def __init__( + self, + project_id=None, + inspect_job=None, + risk_job=None, + job_id=None, + retry=None, + timeout=None, + metadata=None, + wait_until_finished=True, + gcp_conn_id="google_cloud_default", + *args, + **kwargs + ): + super().__init__(*args, **kwargs) + self.project_id = project_id + self.inspect_job = inspect_job + self.risk_job = risk_job + self.job_id = job_id + self.retry = retry + self.timeout = timeout + self.metadata = metadata + self.wait_until_finished = wait_until_finished + self.gcp_conn_id = gcp_conn_id + + def execute(self, context): + hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id) + return hook.create_dlp_job( + project_id=self.project_id, + inspect_job=self.inspect_job, + risk_job=self.risk_job, + job_id=self.job_id, + retry=self.retry, + timeout=self.timeout, + metadata=self.metadata, + wait_until_finished=self.wait_until_finished, + ) + + +class CloudDLPCreateInspectTemplateOperator(BaseOperator): + """ + Creates an InspectTemplate for re-using frequently used configuration for + inspecting content, images, and storage. + + :param organization_id: (Optional) The organization ID. Required to set this + field if parent resource is an organzation. + :type organization_id: str + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. Only set this field if the parent resource is + a project instead of an organzation. + :type project_id: str + :param inspect_template: (Optional) The InspectTemplate to create. + :type inspect_template: dict or google.cloud.dlp_v2.types.InspectTemplate + :param template_id: (Optional) The template ID. + :type template_id: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform. + :type gcp_conn_id: str + :rtype: google.cloud.dlp_v2.types.InspectTemplate + """ + + template_fields = ( + "organization_id", + "project_id", + "inspect_template", + "template_id", + "gcp_conn_id", + ) + + @apply_defaults + def __init__( + self, + organization_id=None, + project_id=None, + inspect_template=None, + template_id=None, + retry=None, + timeout=None, + metadata=None, + gcp_conn_id="google_cloud_default", + *args, + **kwargs + ): + super().__init__(*args, **kwargs) + self.organization_id = organization_id + self.project_id = project_id + self.inspect_template = inspect_template + self.template_id = template_id + self.retry = retry + self.timeout = timeout + self.metadata = metadata + self.gcp_conn_id = gcp_conn_id + + def execute(self, context): + hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id) + return hook.create_inspect_template( + organization_id=self.organization_id, + project_id=self.project_id, + inspect_template=self.inspect_template, + template_id=self.template_id, + retry=self.retry, + timeout=self.timeout, + metadata=self.metadata, + ) + + +class CloudDLPCreateJobTriggerOperator(BaseOperator): + """ + Creates a job trigger to run DLP actions such as scanning storage for sensitive + information on a set schedule. + + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. If set to None or missing, the default + project_id from the GCP connection is used. + :type project_id: str + :param job_trigger: (Optional) The JobTrigger to create. + :type job_trigger: dict or google.cloud.dlp_v2.types.JobTrigger + :param trigger_id: (Optional) The JobTrigger ID. + :type trigger_id: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform. + :type gcp_conn_id: str + :rtype: google.cloud.dlp_v2.types.JobTrigger + """ + + template_fields = ("project_id", "job_trigger", "trigger_id", "gcp_conn_id") + + @apply_defaults + def __init__( + self, + project_id=None, + job_trigger=None, + trigger_id=None, + retry=None, + timeout=None, + metadata=None, + gcp_conn_id="google_cloud_default", + *args, + **kwargs + ): + super().__init__(*args, **kwargs) + self.project_id = project_id + self.job_trigger = job_trigger + self.trigger_id = trigger_id + self.retry = retry + self.timeout = timeout + self.metadata = metadata + self.gcp_conn_id = gcp_conn_id + + def execute(self, context): + hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id) + return hook.create_job_trigger( + project_id=self.project_id, + job_trigger=self.job_trigger, + trigger_id=self.trigger_id, + retry=self.retry, + timeout=self.timeout, + metadata=self.metadata, + ) + + +class CloudDLPCreateStoredInfoTypeOperator(BaseOperator): + """ + Creates a pre-built stored infoType to be used for inspection. + + :param organization_id: (Optional) The organization ID. Required to set this + field if parent resource is an organzation. + :type organization_id: str + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. Only set this field if the parent resource is + a project instead of an organzation. + :type project_id: str + :param config: (Optional) The config for the StoredInfoType. + :type config: dict or google.cloud.dlp_v2.types.StoredInfoTypeConfig + :param stored_info_type_id: (Optional) The StoredInfoType ID. + :type stored_info_type_id: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform. + :type gcp_conn_id: str + :rtype: google.cloud.dlp_v2.types.StoredInfoType + """ + + template_fields = ( + "organization_id", + "project_id", + "config", + "stored_info_type_id", + "gcp_conn_id", + ) + + @apply_defaults + def __init__( + self, + organization_id=None, + project_id=None, + config=None, + stored_info_type_id=None, + retry=None, + timeout=None, + metadata=None, + gcp_conn_id="google_cloud_default", + *args, + **kwargs + ): + super().__init__(*args, **kwargs) + self.organization_id = organization_id + self.project_id = project_id + self.config = config + self.stored_info_type_id = stored_info_type_id + self.retry = retry + self.timeout = timeout + self.metadata = metadata + self.gcp_conn_id = gcp_conn_id + + def execute(self, context): + hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id) + return hook.create_stored_info_type( + organization_id=self.organization_id, + project_id=self.project_id, + config=self.config, + stored_info_type_id=self.stored_info_type_id, + retry=self.retry, + timeout=self.timeout, + metadata=self.metadata, + ) + + +class CloudDLPDeidentifyContentOperator(BaseOperator): + """ + De-identifies potentially sensitive info from a ContentItem. This method has limits + on input size and output size. + + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. If set to None or missing, the default + project_id from the GCP connection is used. + :type project_id: str + :param deidentify_config: (Optional) Configuration for the de-identification of the + content item. Items specified here will override the template referenced by the + deidentify_template_name argument. + :type deidentify_config: dict or google.cloud.dlp_v2.types.DeidentifyConfig + :param inspect_config: (Optional) Configuration for the inspector. Items specified + here will override the template referenced by the inspect_template_name argument. + :type inspect_config: dict or google.cloud.dlp_v2.types.InspectConfig + :param item: (Optional) The item to de-identify. Will be treated as text. + :type item: dict or google.cloud.dlp_v2.types.ContentItem + :param inspect_template_name: (Optional) Optional template to use. Any configuration + directly specified in inspect_config will override those set in the template. + :type inspect_template_name: str + :param deidentify_template_name: (Optional) Optional template to use. Any + configuration directly specified in deidentify_config will override those set + in the template. + :type deidentify_template_name: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform. + :type gcp_conn_id: str + :rtype: google.cloud.dlp_v2.types.DeidentifyContentResponse + """ + + template_fields = ( + "project_id", + "deidentify_config", + "inspect_config", + "item", + "inspect_template_name", + "deidentify_template_name", + "gcp_conn_id", + ) + + @apply_defaults + def __init__( + self, + project_id=None, + deidentify_config=None, + inspect_config=None, + item=None, + inspect_template_name=None, + deidentify_template_name=None, + retry=None, + timeout=None, + metadata=None, + gcp_conn_id="google_cloud_default", + *args, + **kwargs + ): + super().__init__(*args, **kwargs) + self.project_id = project_id + self.deidentify_config = deidentify_config + self.inspect_config = inspect_config + self.item = item + self.inspect_template_name = inspect_template_name + self.deidentify_template_name = deidentify_template_name + self.retry = retry + self.timeout = timeout + self.metadata = metadata + self.gcp_conn_id = gcp_conn_id + + def execute(self, context): + hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id) + return hook.deidentify_content( + project_id=self.project_id, + deidentify_config=self.deidentify_config, + inspect_config=self.inspect_config, + item=self.item, + inspect_template_name=self.inspect_template_name, + deidentify_template_name=self.deidentify_template_name, + retry=self.retry, + timeout=self.timeout, + metadata=self.metadata, + ) + + +class CloudDLPDeleteDeidentifyTemplateOperator(BaseOperator): + """ + Deletes a DeidentifyTemplate. + + :param template_id: The ID of deidentify template to be deleted. + :type template_id: str + :param organization_id: (Optional) The organization ID. Required to set this + field if parent resource is an organzation. + :type organization_id: str + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. Only set this field if the parent resource is + a project instead of an organzation. + :type project_id: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform. + :type gcp_conn_id: str + """ + + template_fields = ("template_id", "organization_id", "project_id", "gcp_conn_id") + + @apply_defaults + def __init__( + self, + template_id, + organization_id=None, + project_id=None, + retry=None, + timeout=None, + metadata=None, + gcp_conn_id="google_cloud_default", + *args, + **kwargs + ): + super().__init__(*args, **kwargs) + self.template_id = template_id + self.organization_id = organization_id + self.project_id = project_id + self.retry = retry + self.timeout = timeout + self.metadata = metadata + self.gcp_conn_id = gcp_conn_id + + def execute(self, context): + hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id) + hook.delete_deidentify_template( + template_id=self.template_id, + organization_id=self.organization_id, + project_id=self.project_id, + retry=self.retry, + timeout=self.timeout, + metadata=self.metadata, + ) + + +class CloudDLPDeleteDlpJobOperator(BaseOperator): + """ + Deletes a long-running DlpJob. This method indicates that the client is no longer + interested in the DlpJob result. The job will be cancelled if possible. + + :param dlp_job_id: The ID of the DLP job resource to be cancelled. + :type dlp_job_id: str + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. If set to None or missing, the default + project_id from the GCP connection is used. + :type project_id: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform. + :type gcp_conn_id: str + """ + + template_fields = ("dlp_job_id", "project_id", "gcp_conn_id") + + @apply_defaults + def __init__( + self, + dlp_job_id, + project_id=None, + retry=None, + timeout=None, + metadata=None, + gcp_conn_id="google_cloud_default", + *args, + **kwargs + ): + super().__init__(*args, **kwargs) + self.dlp_job_id = dlp_job_id + self.project_id = project_id + self.retry = retry + self.timeout = timeout + self.metadata = metadata + self.gcp_conn_id = gcp_conn_id + + def execute(self, context): + hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id) + hook.delete_dlp_job( + dlp_job_id=self.dlp_job_id, + project_id=self.project_id, + retry=self.retry, + timeout=self.timeout, + metadata=self.metadata, + ) + + +class CloudDLPDeleteInspectTemplateOperator(BaseOperator): + """ + Deletes an InspectTemplate. + + :param template_id: The ID of the inspect template to be deleted. + :type template_id: str + :param organization_id: (Optional) The organization ID. Required to set this + field if parent resource is an organzation. + :type organization_id: str + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. Only set this field if the parent resource is + a project instead of an organzation. + :type project_id: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform. + :type gcp_conn_id: str + """ + + template_fields = ("template_id", "organization_id", "project_id", "gcp_conn_id") + + @apply_defaults + def __init__( + self, + template_id, + organization_id=None, + project_id=None, + retry=None, + timeout=None, + metadata=None, + gcp_conn_id="google_cloud_default", + *args, + **kwargs + ): + super().__init__(*args, **kwargs) + self.template_id = template_id + self.organization_id = organization_id + self.project_id = project_id + self.retry = retry + self.timeout = timeout + self.metadata = metadata + self.gcp_conn_id = gcp_conn_id + + def execute(self, context): + hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id) + hook.delete_inspect_template( + template_id=self.template_id, + organization_id=self.organization_id, + project_id=self.project_id, + retry=self.retry, + timeout=self.timeout, + metadata=self.metadata, + ) + + +class CloudDLPDeleteJobTriggerOperator(BaseOperator): + """ + Deletes a job trigger. + + :param job_trigger_id: The ID of the DLP job trigger to be deleted. + :type job_trigger_id: str + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. If set to None or missing, the default + project_id from the GCP connection is used. + :type project_id: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform. + :type gcp_conn_id: str + """ + + template_fields = ("job_trigger_id", "project_id", "gcp_conn_id") + + @apply_defaults + def __init__( + self, + job_trigger_id, + project_id=None, + retry=None, + timeout=None, + metadata=None, + gcp_conn_id="google_cloud_default", + *args, + **kwargs + ): + super().__init__(*args, **kwargs) + self.job_trigger_id = job_trigger_id + self.project_id = project_id + self.retry = retry + self.timeout = timeout + self.metadata = metadata + self.gcp_conn_id = gcp_conn_id + + def execute(self, context): + hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id) + hook.delete_job_trigger( + job_trigger_id=self.job_trigger_id, + project_id=self.project_id, + retry=self.retry, + timeout=self.timeout, + metadata=self.metadata, + ) + + +class CloudDLPDeleteStoredInfoTypeOperator(BaseOperator): + """ + Deletes a stored infoType. + + :param stored_info_type_id: The ID of the stored info type to be deleted. + :type stored_info_type_id: str + :param organization_id: (Optional) The organization ID. Required to set this + field if parent resource is an organzation. + :type organization_id: str + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. Only set this field if the parent resource is + a project instead of an organzation. + :type project_id: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform. + :type gcp_conn_id: str + """ + + template_fields = ( + "stored_info_type_id", + "organization_id", + "project_id", + "gcp_conn_id", + ) + + @apply_defaults + def __init__( + self, + stored_info_type_id, + organization_id=None, + project_id=None, + retry=None, + timeout=None, + metadata=None, + gcp_conn_id="google_cloud_default", + *args, + **kwargs + ): + super().__init__(*args, **kwargs) + self.stored_info_type_id = stored_info_type_id + self.organization_id = organization_id + self.project_id = project_id + self.retry = retry + self.timeout = timeout + self.metadata = metadata + self.gcp_conn_id = gcp_conn_id + + def execute(self, context): + hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id) + hook.delete_stored_info_type( + stored_info_type_id=self.stored_info_type_id, + organization_id=self.organization_id, + project_id=self.project_id, + retry=self.retry, + timeout=self.timeout, + metadata=self.metadata, + ) + + +class CloudDLPGetDeidentifyTemplateOperator(BaseOperator): + """ + Gets a DeidentifyTemplate. + + :param template_id: The ID of deidentify template to be read. + :type template_id: str + :param organization_id: (Optional) The organization ID. Required to set this + field if parent resource is an organzation. + :type organization_id: str + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. Only set this field if the parent resource is + a project instead of an organzation. + :type project_id: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform. + :type gcp_conn_id: str + :rtype: google.cloud.dlp_v2.types.DeidentifyTemplate + """ + + template_fields = ("template_id", "organization_id", "project_id", "gcp_conn_id") + + @apply_defaults + def __init__( + self, + template_id, + organization_id=None, + project_id=None, + retry=None, + timeout=None, + metadata=None, + gcp_conn_id="google_cloud_default", + *args, + **kwargs + ): + super().__init__(*args, **kwargs) + self.template_id = template_id + self.organization_id = organization_id + self.project_id = project_id + self.retry = retry + self.timeout = timeout + self.metadata = metadata + self.gcp_conn_id = gcp_conn_id + + def execute(self, context): + hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id) + return hook.get_deidentify_template( + template_id=self.template_id, + organization_id=self.organization_id, + project_id=self.project_id, + retry=self.retry, + timeout=self.timeout, + metadata=self.metadata, + ) + + +class CloudDLPGetDlpJobOperator(BaseOperator): + """ + Gets the latest state of a long-running DlpJob. + + :param dlp_job_id: The ID of the DLP job resource to be read. + :type dlp_job_id: str + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. If set to None or missing, the default + project_id from the GCP connection is used. + :type project_id: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform. + :type gcp_conn_id: str + :rtype: google.cloud.dlp_v2.types.DlpJob + """ + + template_fields = ("dlp_job_id", "project_id", "gcp_conn_id") + + @apply_defaults + def __init__( + self, + dlp_job_id, + project_id=None, + retry=None, + timeout=None, + metadata=None, + gcp_conn_id="google_cloud_default", + *args, + **kwargs + ): + super().__init__(*args, **kwargs) + self.dlp_job_id = dlp_job_id + self.project_id = project_id + self.retry = retry + self.timeout = timeout + self.metadata = metadata + self.gcp_conn_id = gcp_conn_id + + def execute(self, context): + hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id) + return hook.get_dlp_job( + dlp_job_id=self.dlp_job_id, + project_id=self.project_id, + retry=self.retry, + timeout=self.timeout, + metadata=self.metadata, + ) + + +class CloudDLPGetInspectTemplateOperator(BaseOperator): + """ + Gets an InspectTemplate. + + :param template_id: The ID of inspect template to be read. + :type template_id: str + :param organization_id: (Optional) The organization ID. Required to set this + field if parent resource is an organzation. + :type organization_id: str + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. Only set this field if the parent resource is + a project instead of an organzation. + :type project_id: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform. + :type gcp_conn_id: str + :rtype: google.cloud.dlp_v2.types.InspectTemplate + """ + + template_fields = ("template_id", "organization_id", "project_id", "gcp_conn_id") + + @apply_defaults + def __init__( + self, + template_id, + organization_id=None, + project_id=None, + retry=None, + timeout=None, + metadata=None, + gcp_conn_id="google_cloud_default", + *args, + **kwargs + ): + super().__init__(*args, **kwargs) + self.template_id = template_id + self.organization_id = organization_id + self.project_id = project_id + self.retry = retry + self.timeout = timeout + self.metadata = metadata + self.gcp_conn_id = gcp_conn_id + + def execute(self, context): + hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id) + return hook.get_inspect_template( + template_id=self.template_id, + organization_id=self.organization_id, + project_id=self.project_id, + retry=self.retry, + timeout=self.timeout, + metadata=self.metadata, + ) + + +class CloudDLPGetJobTripperOperator(BaseOperator): + """ + Gets a job trigger. + + :param job_trigger_id: The ID of the DLP job trigger to be read. + :type job_trigger_id: str + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. If set to None or missing, the default + project_id from the GCP connection is used. + :type project_id: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform. + :type gcp_conn_id: str + :rtype: google.cloud.dlp_v2.types.JobTrigger + """ + + template_fields = ("job_trigger_id", "project_id", "gcp_conn_id") + + @apply_defaults + def __init__( + self, + job_trigger_id, + project_id=None, + retry=None, + timeout=None, + metadata=None, + gcp_conn_id="google_cloud_default", + *args, + **kwargs + ): + super().__init__(*args, **kwargs) + self.job_trigger_id = job_trigger_id + self.project_id = project_id + self.retry = retry + self.timeout = timeout + self.metadata = metadata + self.gcp_conn_id = gcp_conn_id + + def execute(self, context): + hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id) + return hook.get_job_trigger( + job_trigger_id=self.job_trigger_id, + project_id=self.project_id, + retry=self.retry, + timeout=self.timeout, + metadata=self.metadata, + ) + + +class CloudDLPGetStoredInfoTypeOperator(BaseOperator): + """ + Gets a stored infoType. + + :param stored_info_type_id: The ID of the stored info type to be read. + :type stored_info_type_id: str + :param organization_id: (Optional) The organization ID. Required to set this + field if parent resource is an organzation. + :type organization_id: str + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. Only set this field if the parent resource is + a project instead of an organzation. + :type project_id: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform. + :type gcp_conn_id: str + :rtype: google.cloud.dlp_v2.types.StoredInfoType + """ + + template_fields = ( + "stored_info_type_id", + "organization_id", + "project_id", + "gcp_conn_id", + ) + + @apply_defaults + def __init__( + self, + stored_info_type_id, + organization_id=None, + project_id=None, + retry=None, + timeout=None, + metadata=None, + gcp_conn_id="google_cloud_default", + *args, + **kwargs + ): + super().__init__(*args, **kwargs) + self.stored_info_type_id = stored_info_type_id + self.organization_id = organization_id + self.project_id = project_id + self.retry = retry + self.timeout = timeout + self.metadata = metadata + self.gcp_conn_id = gcp_conn_id + + def execute(self, context): + hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id) + return hook.get_stored_info_type( + stored_info_type_id=self.stored_info_type_id, + organization_id=self.organization_id, + project_id=self.project_id, + retry=self.retry, + timeout=self.timeout, + metadata=self.metadata, + ) + + +class CloudDLPInspectContentOperator(BaseOperator): + """ + Finds potentially sensitive info in content. This method has limits on + input size, processing time, and output size. + + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. If set to None or missing, the default + project_id from the GCP connection is used. + :type project_id: str + :param inspect_config: (Optional) Configuration for the inspector. Items specified + here will override the template referenced by the inspect_template_name argument. + :type inspect_config: dict or google.cloud.dlp_v2.types.InspectConfig + :param item: (Optional) The item to de-identify. Will be treated as text. + :type item: dict or google.cloud.dlp_v2.types.ContentItem + :param inspect_template_name: (Optional) Optional template to use. Any configuration + directly specified in inspect_config will override those set in the template. + :type inspect_template_name: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform. + :type gcp_conn_id: str + :rtype: google.cloud.tasks_v2.types.InspectContentResponse + """ + + template_fields = ( + "project_id", + "inspect_config", + "item", + "inspect_template_name", + "gcp_conn_id", + ) + + @apply_defaults + def __init__( + self, + project_id=None, + inspect_config=None, + item=None, + inspect_template_name=None, + retry=None, + timeout=None, + metadata=None, + gcp_conn_id="google_cloud_default", + *args, + **kwargs + ): + super().__init__(*args, **kwargs) + self.project_id = project_id + self.inspect_config = inspect_config + self.item = item + self.inspect_template_name = inspect_template_name + self.retry = retry + self.timeout = timeout + self.metadata = metadata + self.gcp_conn_id = gcp_conn_id + + def execute(self, context): + hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id) + return hook.inspect_content( + project_id=self.project_id, + inspect_config=self.inspect_config, + item=self.item, + inspect_template_name=self.inspect_template_name, + retry=self.retry, + timeout=self.timeout, + metadata=self.metadata, + ) + + +class CloudDLPListDeidentifyTemplatesOperator(BaseOperator): + """ + Lists DeidentifyTemplates. + + :param organization_id: (Optional) The organization ID. Required to set this + field if parent resource is an organzation. + :type organization_id: str + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. Only set this field if the parent resource is + a project instead of an organzation. + :type project_id: str + :param page_size: (Optional) The maximum number of resources contained in the + underlying API response. + :type page_size: int + :param order_by: (Optional) Optional comma separated list of fields to order by, + followed by asc or desc postfix. + :type order_by: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform. + :type gcp_conn_id: str + :rtype: list[google.cloud.dlp_v2.types.DeidentifyTemplate] + """ + + template_fields = ("organization_id", "project_id", "gcp_conn_id") + + @apply_defaults + def __init__( + self, + organization_id=None, + project_id=None, + page_size=None, + order_by=None, + retry=None, + timeout=None, + metadata=None, + gcp_conn_id="google_cloud_default", + *args, + **kwargs + ): + super().__init__(*args, **kwargs) + self.organization_id = organization_id + self.project_id = project_id + self.page_size = page_size + self.order_by = order_by + self.retry = retry + self.timeout = timeout + self.metadata = metadata + self.gcp_conn_id = gcp_conn_id + + def execute(self, context): + hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id) + return hook.list_deidentify_templates( + organization_id=self.organization_id, + project_id=self.project_id, + page_size=self.page_size, + order_by=self.order_by, + retry=self.retry, + timeout=self.timeout, + metadata=self.metadata, + ) + + +class CloudDLPListDlpJobsOperator(BaseOperator): + """ + Lists DlpJobs that match the specified filter in the request. + + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. If set to None or missing, the default + project_id from the GCP connection is used. + :type project_id: str + :param results_filter: (Optional) Filter used to specify a subset of results. + :type results_filter: str + :param page_size: (Optional) The maximum number of resources contained in the + underlying API response. + :type page_size: int + :param job_type: (Optional) The type of job. + :type job_type: str + :param order_by: (Optional) Optional comma separated list of fields to order by, + followed by asc or desc postfix. + :type order_by: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform. + :type gcp_conn_id: str + :rtype: list[google.cloud.dlp_v2.types.DlpJob] + """ + + template_fields = ("project_id", "gcp_conn_id") + + @apply_defaults + def __init__( + self, + project_id=None, + results_filter=None, + page_size=None, + job_type=None, + order_by=None, + retry=None, + timeout=None, + metadata=None, + gcp_conn_id="google_cloud_default", + *args, + **kwargs + ): + super().__init__(*args, **kwargs) + self.project_id = project_id + self.results_filter = results_filter + self.page_size = page_size + self.job_type = job_type + self.order_by = order_by + self.retry = retry + self.timeout = timeout + self.metadata = metadata + self.gcp_conn_id = gcp_conn_id + + def execute(self, context): + hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id) + return hook.list_dlp_jobs( + project_id=self.project_id, + results_filter=self.results_filter, + page_size=self.page_size, + job_type=self.job_type, + order_by=self.order_by, + retry=self.retry, + timeout=self.timeout, + metadata=self.metadata, + ) + + +class CloudDLPListInfoTypesOperator(BaseOperator): + """ + Returns a list of the sensitive information types that the DLP API supports. + + :param language_code: (Optional) Optional BCP-47 language code for localized infoType + friendly names. If omitted, or if localized strings are not available, en-US + strings will be returned. + :type language_code: str + :param results_filter: (Optional) Filter used to specify a subset of results. + :type results_filter: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform. + :type gcp_conn_id: str + :rtype: ListInfoTypesResponse + """ + + template_fields = ("language_code", "gcp_conn_id") + + @apply_defaults + def __init__( + self, + language_code=None, + results_filter=None, + retry=None, + timeout=None, + metadata=None, + gcp_conn_id="google_cloud_default", + *args, + **kwargs + ): + super().__init__(*args, **kwargs) + self.language_code = language_code + self.results_filter = results_filter + self.retry = retry + self.timeout = timeout + self.metadata = metadata + self.gcp_conn_id = gcp_conn_id + + def execute(self, context): + hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id) + return hook.list_info_types( + language_code=self.language_code, + results_filter=self.results_filter, + retry=self.retry, + timeout=self.timeout, + metadata=self.metadata, + ) + + +class CloudDLPListInspectTemplatesOperator(BaseOperator): + """ + Lists InspectTemplates. + + :param organization_id: (Optional) The organization ID. Required to set this + field if parent resource is an organzation. + :type organization_id: str + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. Only set this field if the parent resource is + a project instead of an organzation. + :type project_id: str + :param page_size: (Optional) The maximum number of resources contained in the + underlying API response. + :type page_size: int + :param order_by: (Optional) Optional comma separated list of fields to order by, + followed by asc or desc postfix. + :type order_by: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform. + :type gcp_conn_id: str + :rtype: list[google.cloud.dlp_v2.types.InspectTemplate] + """ + + template_fields = ("organization_id", "project_id", "gcp_conn_id") + + @apply_defaults + def __init__( + self, + organization_id=None, + project_id=None, + page_size=None, + order_by=None, + retry=None, + timeout=None, + metadata=None, + gcp_conn_id="google_cloud_default", + *args, + **kwargs + ): + super().__init__(*args, **kwargs) + self.organization_id = organization_id + self.project_id = project_id + self.page_size = page_size + self.order_by = order_by + self.retry = retry + self.timeout = timeout + self.metadata = metadata + self.gcp_conn_id = gcp_conn_id + + def execute(self, context): + hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id) + return hook.list_inspect_templates( + organization_id=self.organization_id, + project_id=self.project_id, + page_size=self.page_size, + order_by=self.order_by, + retry=self.retry, + timeout=self.timeout, + metadata=self.metadata, + ) + + +class CloudDLPListJobTriggersOperator(BaseOperator): + """ + Lists job triggers. + + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. If set to None or missing, the default + project_id from the GCP connection is used. + :type project_id: str + :param page_size: (Optional) The maximum number of resources contained in the + underlying API response. + :type page_size: int + :param order_by: (Optional) Optional comma separated list of fields to order by, + followed by asc or desc postfix. + :type order_by: str + :param results_filter: (Optional) Filter used to specify a subset of results. + :type results_filter: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform. + :type gcp_conn_id: str + :rtype: list[google.cloud.dlp_v2.types.JobTrigger] + """ + + template_fields = ("project_id", "gcp_conn_id") + + @apply_defaults + def __init__( + self, + project_id=None, + page_size=None, + order_by=None, + results_filter=None, + retry=None, + timeout=None, + metadata=None, + gcp_conn_id="google_cloud_default", + *args, + **kwargs + ): + super().__init__(*args, **kwargs) + self.project_id = project_id + self.page_size = page_size + self.order_by = order_by + self.results_filter = results_filter + self.retry = retry + self.timeout = timeout + self.metadata = metadata + self.gcp_conn_id = gcp_conn_id + + def execute(self, context): + hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id) + return hook.list_job_triggers( + project_id=self.project_id, + page_size=self.page_size, + order_by=self.order_by, + results_filter=self.results_filter, + retry=self.retry, + timeout=self.timeout, + metadata=self.metadata, + ) + + +class CloudDLPListStoredInfoTypesOperator(BaseOperator): + """ + Lists stored infoTypes. + + :param organization_id: (Optional) The organization ID. Required to set this + field if parent resource is an organzation. + :type organization_id: str + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. Only set this field if the parent resource is + a project instead of an organzation. + :type project_id: str + :param page_size: (Optional) The maximum number of resources contained in the + underlying API response. + :type page_size: int + :param order_by: (Optional) Optional comma separated list of fields to order by, + followed by asc or desc postfix. + :type order_by: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform. + :type gcp_conn_id: str + :rtype: list[google.cloud.dlp_v2.types.StoredInfoType] + """ + + template_fields = ("organization_id", "project_id", "gcp_conn_id") + + @apply_defaults + def __init__( + self, + organization_id=None, + project_id=None, + page_size=None, + order_by=None, + retry=None, + timeout=None, + metadata=None, + gcp_conn_id="google_cloud_default", + *args, + **kwargs + ): + super().__init__(*args, **kwargs) + self.organization_id = organization_id + self.project_id = project_id + self.page_size = page_size + self.order_by = order_by + self.retry = retry + self.timeout = timeout + self.metadata = metadata + self.gcp_conn_id = gcp_conn_id + + def execute(self, context): + hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id) + return hook.list_stored_info_types( + organization_id=self.organization_id, + project_id=self.project_id, + page_size=self.page_size, + order_by=self.order_by, + retry=self.retry, + timeout=self.timeout, + metadata=self.metadata, + ) + + +class CloudDLPRedactImageOperator(BaseOperator): + """ + Redacts potentially sensitive info from an image. This method has limits on + input size, processing time, and output size. + + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. If set to None or missing, the default + project_id from the GCP connection is used. + :type project_id: str + :param inspect_config: (Optional) Configuration for the inspector. Items specified + here will override the template referenced by the inspect_template_name argument. + :type inspect_config: dict or google.cloud.dlp_v2.types.InspectConfig + :param image_redaction_configs: (Optional) The configuration for specifying what + content to redact from images. + :type image_redaction_configs: list[dict] or list[google.cloud.dlp_v2.types.ImageRedactionConfig] + :param include_findings: (Optional) Whether the response should include findings + along with the redacted image. + :type include_findings: bool + :param byte_item: (Optional) The content must be PNG, JPEG, SVG or BMP. + :type byte_item: dict or google.cloud.dlp_v2.types.ByteContentItem + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform. + :type gcp_conn_id: str + :rtype: google.cloud.dlp_v2.types.RedactImageResponse + """ + + template_fields = ( + "project_id", + "inspect_config", + "image_redaction_configs", + "include_findings", + "byte_item", + "gcp_conn_id", + ) + + @apply_defaults + def __init__( + self, + project_id=None, + inspect_config=None, + image_redaction_configs=None, + include_findings=None, + byte_item=None, + retry=None, + timeout=None, + metadata=None, + gcp_conn_id="google_cloud_default", + *args, + **kwargs + ): + super().__init__(*args, **kwargs) + self.project_id = project_id + self.inspect_config = inspect_config + self.image_redaction_configs = image_redaction_configs + self.include_findings = include_findings + self.byte_item = byte_item + self.retry = retry + self.timeout = timeout + self.metadata = metadata + self.gcp_conn_id = gcp_conn_id + + def execute(self, context): + hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id) + return hook.redact_image( + project_id=self.project_id, + inspect_config=self.inspect_config, + image_redaction_configs=self.image_redaction_configs, + include_findings=self.include_findings, + byte_item=self.byte_item, + retry=self.retry, + timeout=self.timeout, + metadata=self.metadata, + ) + + +class CloudDLPReidentifyContentOperator(BaseOperator): + """ + Re-identifies content that has been de-identified. + + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. If set to None or missing, the default + project_id from the GCP connection is used. + :type project_id: str + :param reidentify_config: (Optional) Configuration for the re-identification of + the content item. + :type reidentify_config: dict or google.cloud.dlp_v2.types.DeidentifyConfig + :param inspect_config: (Optional) Configuration for the inspector. + :type inspect_config: dict or google.cloud.dlp_v2.types.InspectConfig + :param item: (Optional) The item to re-identify. Will be treated as text. + :type item: dict or google.cloud.dlp_v2.types.ContentItem + :param inspect_template_name: (Optional) Optional template to use. Any configuration + directly specified in inspect_config will override those set in the template. + :type inspect_template_name: str + :param reidentify_template_name: (Optional) Optional template to use. References an + instance of DeidentifyTemplate. Any configuration directly specified in + reidentify_config or inspect_config will override those set in the template. + :type reidentify_template_name: str + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform. + :type gcp_conn_id: str + :rtype: google.cloud.dlp_v2.types.ReidentifyContentResponse + """ + + template_fields = ( + "project_id", + "reidentify_config", + "inspect_config", + "item", + "inspect_template_name", + "reidentify_template_name", + "gcp_conn_id", + ) + + @apply_defaults + def __init__( + self, + project_id=None, + reidentify_config=None, + inspect_config=None, + item=None, + inspect_template_name=None, + reidentify_template_name=None, + retry=None, + timeout=None, + metadata=None, + gcp_conn_id="google_cloud_default", + *args, + **kwargs + ): + super().__init__(*args, **kwargs) + self.project_id = project_id + self.reidentify_config = reidentify_config + self.inspect_config = inspect_config + self.item = item + self.inspect_template_name = inspect_template_name + self.reidentify_template_name = reidentify_template_name + self.retry = retry + self.timeout = timeout + self.metadata = metadata + self.gcp_conn_id = gcp_conn_id + + def execute(self, context): + hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id) + return hook.reidentify_content( + project_id=self.project_id, + reidentify_config=self.reidentify_config, + inspect_config=self.inspect_config, + item=self.item, + inspect_template_name=self.inspect_template_name, + reidentify_template_name=self.reidentify_template_name, + retry=self.retry, + timeout=self.timeout, + metadata=self.metadata, + ) + + +class CloudDLPUpdateDeidentifyTemplateOperator(BaseOperator): + """ + Updates the DeidentifyTemplate. + + :param template_id: The ID of deidentify template to be updated. + :type template_id: str + :param organization_id: (Optional) The organization ID. Required to set this + field if parent resource is an organzation. + :type organization_id: str + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. Only set this field if the parent resource is + a project instead of an organzation. + :type project_id: str + :param deidentify_template: New DeidentifyTemplate value. + :type deidentify_template: dict or google.cloud.dlp_v2.types.DeidentifyTemplate + :param update_mask: Mask to control which fields get updated. + :type update_mask: dict or google.cloud.dlp_v2.types.FieldMask + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform. + :type gcp_conn_id: str + :rtype: google.cloud.dlp_v2.types.DeidentifyTemplate + """ + + template_fields = ( + "template_id", + "organization_id", + "project_id", + "deidentify_template", + "update_mask", + "gcp_conn_id", + ) + + @apply_defaults + def __init__( + self, + template_id, + organization_id=None, + project_id=None, + deidentify_template=None, + update_mask=None, + retry=None, + timeout=None, + metadata=None, + gcp_conn_id="google_cloud_default", + *args, + **kwargs + ): + super().__init__(*args, **kwargs) + self.template_id = template_id + self.organization_id = organization_id + self.project_id = project_id + self.deidentify_template = deidentify_template + self.update_mask = update_mask + self.retry = retry + self.timeout = timeout + self.metadata = metadata + self.gcp_conn_id = gcp_conn_id + + def execute(self, context): + hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id) + return hook.update_deidentify_template( + template_id=self.template_id, + organization_id=self.organization_id, + project_id=self.project_id, + deidentify_template=self.deidentify_template, + update_mask=self.update_mask, + retry=self.retry, + timeout=self.timeout, + metadata=self.metadata, + ) + + +class CloudDLPUpdateInspectTemplateOperator(BaseOperator): + """ + Updates the InspectTemplate. + + :param template_id: The ID of the inspect template to be updated. + :type template_id: str + :param organization_id: (Optional) The organization ID. Required to set this + field if parent resource is an organzation. + :type organization_id: str + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. Only set this field if the parent resource is + a project instead of an organzation. + :type project_id: str + :param inspect_template: New InspectTemplate value. + :type inspect_template: dict or google.cloud.dlp_v2.types.InspectTemplate + :param update_mask: Mask to control which fields get updated. + :type update_mask: dict or google.cloud.dlp_v2.types.FieldMask + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform. + :type gcp_conn_id: str + :rtype: google.cloud.dlp_v2.types.InspectTemplate + """ + + template_fields = ( + "template_id", + "organization_id", + "project_id", + "inspect_template", + "update_mask", + "gcp_conn_id", + ) + + @apply_defaults + def __init__( + self, + template_id, + organization_id=None, + project_id=None, + inspect_template=None, + update_mask=None, + retry=None, + timeout=None, + metadata=None, + gcp_conn_id="google_cloud_default", + *args, + **kwargs + ): + super().__init__(*args, **kwargs) + self.template_id = template_id + self.organization_id = organization_id + self.project_id = project_id + self.inspect_template = inspect_template + self.update_mask = update_mask + self.retry = retry + self.timeout = timeout + self.metadata = metadata + self.gcp_conn_id = gcp_conn_id + + def execute(self, context): + hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id) + return hook.update_inspect_template( + template_id=self.template_id, + organization_id=self.organization_id, + project_id=self.project_id, + inspect_template=self.inspect_template, + update_mask=self.update_mask, + retry=self.retry, + timeout=self.timeout, + metadata=self.metadata, + ) + + +class CloudDLPUpdateJobTriggerOperator(BaseOperator): + """ + Updates a job trigger. + + :param job_trigger_id: The ID of the DLP job trigger to be updated. + :type job_trigger_id: str + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. If set to None or missing, the default + project_id from the GCP connection is used. + :type project_id: str + :param job_trigger: New JobTrigger value. + :type job_trigger: dict or google.cloud.dlp_v2.types.JobTrigger + :param update_mask: Mask to control which fields get updated. + :type update_mask: dict or google.cloud.dlp_v2.types.FieldMask + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform. + :type gcp_conn_id: str + :rtype: google.cloud.dlp_v2.types.InspectTemplate + """ + + template_fields = ( + "job_trigger_id", + "project_id", + "job_trigger", + "update_mask", + "gcp_conn_id", + ) + + @apply_defaults + def __init__( + self, + job_trigger_id, + project_id=None, + job_trigger=None, + update_mask=None, + retry=None, + timeout=None, + metadata=None, + gcp_conn_id="google_cloud_default", + *args, + **kwargs + ): + super().__init__(*args, **kwargs) + self.job_trigger_id = job_trigger_id + self.project_id = project_id + self.job_trigger = job_trigger + self.update_mask = update_mask + self.retry = retry + self.timeout = timeout + self.metadata = metadata + self.gcp_conn_id = gcp_conn_id + + def execute(self, context): + hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id) + return hook.update_job_trigger( + job_trigger_id=self.job_trigger_id, + project_id=self.project_id, + job_trigger=self.job_trigger, + update_mask=self.update_mask, + retry=self.retry, + timeout=self.timeout, + metadata=self.metadata, + ) + + +class CloudDLPUpdateStoredInfoTypeOperator(BaseOperator): + """ + Updates the stored infoType by creating a new version. + + :param stored_info_type_id: The ID of the stored info type to be updated. + :type stored_info_type_id: str + :param organization_id: (Optional) The organization ID. Required to set this + field if parent resource is an organzation. + :type organization_id: str + :param project_id: (Optional) Google Cloud Platform project ID where the + DLP Instance exists. Only set this field if the parent resource is + a project instead of an organzation. + :type project_id: str + :param config: Updated configuration for the storedInfoType. If not provided, a new + version of the storedInfoType will be created with the existing configuration. + :type config: dict or google.cloud.dlp_v2.types.StoredInfoTypeConfig + :param update_mask: Mask to control which fields get updated. + :type update_mask: dict or google.cloud.dlp_v2.types.FieldMask + :param retry: (Optional) A retry object used to retry requests. + If None is specified, requests will not be retried. + :type retry: google.api_core.retry.Retry + :param timeout: (Optional) The amount of time, in seconds, to wait for the request + to complete. Note that if retry is specified, the timeout applies to each + individual attempt. + :type timeout: float + :param metadata: (Optional) Additional metadata that is provided to the method. + :type metadata: sequence[tuple[str, str]]] + :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud Platform. + :type gcp_conn_id: str + :rtype: google.cloud.dlp_v2.types.StoredInfoType + """ + + template_fields = ( + "stored_info_type_id", + "organization_id", + "project_id", + "config", + "update_mask", + "gcp_conn_id", + ) + + @apply_defaults + def __init__( + self, + stored_info_type_id, + organization_id=None, + project_id=None, + config=None, + update_mask=None, + retry=None, + timeout=None, + metadata=None, + gcp_conn_id="google_cloud_default", + *args, + **kwargs + ): + super().__init__(*args, **kwargs) + self.stored_info_type_id = stored_info_type_id + self.organization_id = organization_id + self.project_id = project_id + self.config = config + self.update_mask = update_mask + self.retry = retry + self.timeout = timeout + self.metadata = metadata + self.gcp_conn_id = gcp_conn_id + + def execute(self, context): + hook = CloudDLPHook(gcp_conn_id=self.gcp_conn_id) + return hook.update_stored_info_type( + stored_info_type_id=self.stored_info_type_id, + organization_id=self.organization_id, + project_id=self.project_id, + config=self.config, + update_mask=self.update_mask, + retry=self.retry, + timeout=self.timeout, + metadata=self.metadata, + ) diff --git a/docs/integration.rst b/docs/integration.rst index ee3e734513f03..172964f465640 100644 --- a/docs/integration.rst +++ b/docs/integration.rst @@ -777,6 +777,109 @@ Google Natural Language They also use :class:`airflow.contrib.hooks.gcp_natural_language_operator.CloudNaturalLanguageHook` to communicate with Google Cloud Platform. +Google Cloud Data Loss Prevention (DLP) +''''''''''''''''''''''''''''''''''''''' + +:class:`airflow.contrib.operators.gcp_dlp_operator.CloudDLPCancelDLPJobOperator` + Starts asynchronous cancellation on a long-running DlpJob. + +:class:`airflow.contrib.operators.gcp_dlp_operator.CloudDLPCreateDeidentifyTemplateOperator` + Creates a DeidentifyTemplate for re-using frequently used configuration for + de-identifying content, images, and storage. + +:class:`airflow.contrib.operators.gcp_dlp_operator.CloudDLPCreateDLPJobOperator` + Creates a new job to inspect storage or calculate risk metrics. + +:class:`airflow.contrib.operators.gcp_dlp_operator.CloudDLPCreateInspectTemplateOperator` + Creates an InspectTemplate for re-using frequently used configuration for + inspecting content, images, and storage. + +:class:`airflow.contrib.operators.gcp_dlp_operator.CloudDLPCreateJobTriggerOperator` + Creates a job trigger to run DLP actions such as scanning storage for sensitive + information on a set schedule. + +:class:`airflow.contrib.operators.gcp_dlp_operator.CloudDLPCreateStoredInfoTypeOperator` + Creates a pre-built stored infoType to be used for inspection. + +:class:`airflow.contrib.operators.gcp_dlp_operator.CloudDLPDeidentifyContentOperator` + De-identifies potentially sensitive info from a ContentItem. This method has limits + on input size and output size. + +:class:`airflow.contrib.operators.gcp_dlp_operator.CloudDLPDeleteDeidentifyTemplateOperator` + Deletes a DeidentifyTemplate. + +:class:`airflow.contrib.operators.gcp_dlp_operator.CloudDLPDeleteDlpJobOperator` + Deletes a long-running DlpJob. This method indicates that the client is no longer + interested in the DlpJob result. The job will be cancelled if possible. + +:class:`airflow.contrib.operators.gcp_dlp_operator.CloudDLPDeleteInspectTemplateOperator` + Deletes an InspectTemplate. + +:class:`airflow.contrib.operators.gcp_dlp_operator.CloudDLPDeleteJobTriggerOperator` + Deletes a job trigger. + +:class:`airflow.contrib.operators.gcp_dlp_operator.CloudDLPDeleteStoredInfoTypeOperator` + Deletes a stored infoType. + +:class:`airflow.contrib.operators.gcp_dlp_operator.CloudDLPGetDeidentifyTemplateOperator` + Gets a DeidentifyTemplate. + +:class:`airflow.contrib.operators.gcp_dlp_operator.CloudDLPGetDlpJobOperator` + Gets the latest state of a long-running DlpJob. + +:class:`airflow.contrib.operators.gcp_dlp_operator.CloudDLPGetInspectTemplateOperator` + Gets an InspectTemplate. + +:class:`airflow.contrib.operators.gcp_dlp_operator.CloudDLPGetJobTripperOperator` + Gets a job trigger. + +:class:`airflow.contrib.operators.gcp_dlp_operator.CloudDLPGetStoredInfoTypeOperator` + Gets a stored infoType. + +:class:`airflow.contrib.operators.gcp_dlp_operator.CloudDLPInspectContentOperator` + Finds potentially sensitive info in content. This method has limits on + input size, processing time, and output size. + +:class:`airflow.contrib.operators.gcp_dlp_operator.CloudDLPListDeidentifyTemplatesOperator` + Lists DeidentifyTemplates. + +:class:`airflow.contrib.operators.gcp_dlp_operator.CloudDLPListDlpJobsOperator` + Lists DlpJobs that match the specified filter in the request. + +:class:`airflow.contrib.operators.gcp_dlp_operator.CloudDLPListInfoTypesOperator` + Returns a list of the sensitive information types that the DLP API supports. + +:class:`airflow.contrib.operators.gcp_dlp_operator.CloudDLPListInspectTemplatesOperator` + Lists InspectTemplates. + +:class:`airflow.contrib.operators.gcp_dlp_operator.CloudDLPListJobTriggersOperator` + Lists job triggers. + +:class:`airflow.contrib.operators.gcp_dlp_operator.CloudDLPListStoredInfoTypesOperator` + Lists stored infoTypes. + +:class:`airflow.contrib.operators.gcp_dlp_operator.CloudDLPRedactImageOperator` + Redacts potentially sensitive info from an image. This method has limits on + input size, processing time, and output size. + +:class:`airflow.contrib.operators.gcp_dlp_operator.CloudDLPReidentifyContentOperator` + Re-identifies content that has been de-identified. + +:class:`airflow.contrib.operators.gcp_dlp_operator.CloudDLPUpdateDeidentifyTemplateOperator` + Updates the DeidentifyTemplate. + +:class:`airflow.contrib.operators.gcp_dlp_operator.CloudDLPUpdateInspectTemplateOperator` + Updates the InspectTemplate. + +:class:`airflow.contrib.operators.gcp_dlp_operator.CloudDLPUpdateJobTriggerOperator` + Updates a job trigger. + +:class:`airflow.contrib.operators.gcp_dlp_operator.CloudDLPUpdateStoredInfoTypeOperator` + Updates the stored infoType by creating a new version. + +They also use :class:`airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook` to communicate with Google Cloud Platform. + + .. _Qubole: Qubole diff --git a/setup.py b/setup.py index 9e5a7dbfcf548..5ee6d3dbee7ab 100644 --- a/setup.py +++ b/setup.py @@ -193,6 +193,7 @@ def write_version(filename: str = os.path.join(*["airflow", "git_version"])): 'google-auth>=1.0.0, <2.0.0dev', 'google-cloud-bigtable==0.33.0', 'google-cloud-container>=0.1.1', + 'google-cloud-dlp>=0.11.0', 'google-cloud-language>=1.1.1', 'google-cloud-spanner>=1.7.1', 'google-cloud-storage~=1.16', diff --git a/tests/contrib/hooks/test_gcp_dlp_hook.py b/tests/contrib/hooks/test_gcp_dlp_hook.py new file mode 100644 index 0000000000000..f6f4cd678b4d4 --- /dev/null +++ b/tests/contrib/hooks/test_gcp_dlp_hook.py @@ -0,0 +1,1185 @@ +# -*- coding: utf-8 -*- +# +# 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. + +# pylint: disable=R0904, C0111, C0302 +""" +This module contains various unit tests for +functions in CloudDLPHook +""" + +import unittest +from typing import Any, Dict +from google.cloud.dlp_v2.types import DlpJob + +from airflow import AirflowException +from airflow.contrib.hooks.gcp_dlp_hook import CloudDLPHook +from tests.compat import mock +from tests.contrib.utils.base_gcp_mock import mock_base_gcp_hook_no_default_project_id + +API_RESPONSE = {} # type: Dict[Any, Any] +ORGANIZATION_ID = "test-org" +ORGANIZATION_PATH = "organizations/{}".format(ORGANIZATION_ID) +PROJECT_ID = "test-project" +PROJECT_PATH = "projects/{}".format(PROJECT_ID) +DLP_JOB_ID = "job123" +DLP_JOB_PATH = "projects/{}/dlpJobs/{}".format(PROJECT_ID, DLP_JOB_ID) +TEMPLATE_ID = "template123" +STORED_INFO_TYPE_ID = "type123" +TRIGGER_ID = "trigger123" +DEIDENTIFY_TEMPLATE_ORGANIZATION_PATH = "organizations/{}/deidentifyTemplates/{}".format( + ORGANIZATION_ID, TEMPLATE_ID +) +INSPECT_TEMPLATE_ORGANIZATION_PATH = "organizations/{}/inspectTemplates/{}".format( + ORGANIZATION_ID, TEMPLATE_ID +) +STORED_INFO_TYPE_ORGANIZATION_PATH = "organizations/{}/storedInfoTypes/{}".format( + ORGANIZATION_ID, STORED_INFO_TYPE_ID +) +DEIDENTIFY_TEMPLATE_PROJECT_PATH = "projects/{}/deidentifyTemplates/{}".format( + PROJECT_ID, TEMPLATE_ID +) +INSPECT_TEMPLATE_PROJECT_PATH = "projects/{}/inspectTemplates/{}".format( + PROJECT_ID, TEMPLATE_ID +) +STORED_INFO_TYPE_PROJECT_PATH = "projects/{}/storedInfoTypes/{}".format( + PROJECT_ID, STORED_INFO_TYPE_ID +) +JOB_TRIGGER_PATH = "projects/{}/jobTriggers/{}".format(PROJECT_ID, TRIGGER_ID) + + +class TestCloudDLPHook(unittest.TestCase): + def setUp(self): + with mock.patch( + "airflow.contrib.hooks." "gcp_api_base_hook.GoogleCloudBaseHook.__init__", + new=mock_base_gcp_hook_no_default_project_id, + ): + self.hook = CloudDLPHook(gcp_conn_id="test") + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_cancel_dlp_job(self, get_conn): + self.hook.cancel_dlp_job(dlp_job_id=DLP_JOB_ID, project_id=PROJECT_ID) + + get_conn.return_value.cancel_dlp_job.assert_called_once_with( + name=DLP_JOB_PATH, retry=None, timeout=None, metadata=None + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_cancel_dlp_job_without_dlp_job_id(self, _): + with self.assertRaises(AirflowException): + self.hook.cancel_dlp_job(dlp_job_id=None, project_id=PROJECT_ID) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_cancel_dlp_job_without_parent(self, _): + with self.assertRaises(AirflowException): + self.hook.cancel_dlp_job(dlp_job_id=DLP_JOB_ID) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn", + **{ + "return_value.create_deidentify_template.return_value": API_RESPONSE + }, # type: ignore + ) + def test_create_deidentify_template_with_org_id(self, get_conn): + result = self.hook.create_deidentify_template(organization_id=ORGANIZATION_ID) + + self.assertIs(result, API_RESPONSE) + get_conn.return_value.create_deidentify_template.assert_called_once_with( + parent=ORGANIZATION_PATH, + deidentify_template=None, + template_id=None, + retry=None, + timeout=None, + metadata=None, + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn", + **{ + "return_value.create_deidentify_template.return_value": API_RESPONSE + }, # type: ignore + ) + def test_create_deidentify_template_with_project_id(self, get_conn): + result = self.hook.create_deidentify_template(project_id=PROJECT_ID) + + self.assertIs(result, API_RESPONSE) + get_conn.return_value.create_deidentify_template.assert_called_once_with( + parent=PROJECT_PATH, + deidentify_template=None, + template_id=None, + retry=None, + timeout=None, + metadata=None, + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_create_deidentify_template_without_parent(self, _): + with self.assertRaises(AirflowException): + self.hook.create_deidentify_template() + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn", + **{"return_value.create_dlp_job.return_value": API_RESPONSE}, # type: ignore + ) + def test_create_dlp_job(self, get_conn): + result = self.hook.create_dlp_job( + project_id=PROJECT_ID, wait_until_finished=False + ) + + self.assertIs(result, API_RESPONSE) + get_conn.return_value.create_dlp_job.assert_called_once_with( + parent=PROJECT_PATH, + inspect_job=None, + risk_job=None, + job_id=None, + retry=None, + timeout=None, + metadata=None, + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_create_dlp_job_without_project_id(self, _): + with self.assertRaises(AirflowException): + self.hook.create_dlp_job() + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_create_dlp_job_with_wait_until_finished(self, get_conn): + job_for_create = DlpJob(name=DLP_JOB_PATH, state=DlpJob.JobState.PENDING) + get_conn.return_value.create_dlp_job.return_value = job_for_create + job_for_get = DlpJob(name=DLP_JOB_PATH, state=DlpJob.JobState.DONE) + get_conn.return_value.get_dlp_job.return_value = job_for_get + + self.hook.create_dlp_job(project_id=PROJECT_ID) + + get_conn.return_value.get_dlp_job.assert_called_once_with( + name=DLP_JOB_PATH, retry=None, timeout=None, metadata=None + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn", + **{ + "return_value.create_inspect_template.return_value": API_RESPONSE + }, # type: ignore + ) + def test_create_inspect_template_with_org_id(self, get_conn): + result = self.hook.create_inspect_template(organization_id=ORGANIZATION_ID) + + self.assertIs(result, API_RESPONSE) + get_conn.return_value.create_inspect_template.assert_called_once_with( + parent=ORGANIZATION_PATH, + inspect_template=None, + template_id=None, + retry=None, + timeout=None, + metadata=None, + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn", + **{ + "return_value.create_inspect_template.return_value": API_RESPONSE + }, # type: ignore + ) + def test_create_inspect_template_with_project_id(self, get_conn): + result = self.hook.create_inspect_template(project_id=PROJECT_ID) + + self.assertIs(result, API_RESPONSE) + get_conn.return_value.create_inspect_template.assert_called_once_with( + parent=PROJECT_PATH, + inspect_template=None, + template_id=None, + retry=None, + timeout=None, + metadata=None, + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_create_inspect_template_without_parent(self, _): + with self.assertRaises(AirflowException): + self.hook.create_inspect_template() + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn", + **{ + "return_value.create_job_trigger.return_value": API_RESPONSE + }, # type: ignore + ) + def test_create_job_trigger(self, get_conn): + result = self.hook.create_job_trigger(project_id=PROJECT_ID) + + self.assertIs(result, API_RESPONSE) + get_conn.return_value.create_job_trigger.assert_called_once_with( + parent=PROJECT_PATH, + job_trigger=None, + trigger_id=None, + retry=None, + timeout=None, + metadata=None, + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_create_job_trigger_without_parent(self, _): + with self.assertRaises(AirflowException): + self.hook.create_job_trigger() + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn", + **{ + "return_value.create_stored_info_type.return_value": API_RESPONSE + }, # type: ignore + ) + def test_create_stored_info_type_with_org_id(self, get_conn): + result = self.hook.create_stored_info_type(organization_id=ORGANIZATION_ID) + + self.assertIs(result, API_RESPONSE) + get_conn.return_value.create_stored_info_type.assert_called_once_with( + parent=ORGANIZATION_PATH, + config=None, + stored_info_type_id=None, + retry=None, + timeout=None, + metadata=None, + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn", + **{ + "return_value.create_stored_info_type.return_value": API_RESPONSE + }, # type: ignore + ) + def test_create_stored_info_type_with_project_id(self, get_conn): + result = self.hook.create_stored_info_type(project_id=PROJECT_ID) + + self.assertIs(result, API_RESPONSE) + get_conn.return_value.create_stored_info_type.assert_called_once_with( + parent=PROJECT_PATH, + config=None, + stored_info_type_id=None, + retry=None, + timeout=None, + metadata=None, + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_create_stored_info_type_without_parent(self, _): + with self.assertRaises(AirflowException): + self.hook.create_stored_info_type() + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn", + **{ + "return_value.deidentify_content.return_value": API_RESPONSE + }, # type: ignore + ) + def test_deidentify_content(self, get_conn): + result = self.hook.deidentify_content(project_id=PROJECT_ID) + + self.assertIs(result, API_RESPONSE) + get_conn.return_value.deidentify_content.assert_called_once_with( + parent=PROJECT_PATH, + deidentify_config=None, + inspect_config=None, + item=None, + inspect_template_name=None, + deidentify_template_name=None, + retry=None, + timeout=None, + metadata=None, + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_deidentify_content_without_parent(self, _): + with self.assertRaises(AirflowException): + self.hook.deidentify_content() + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_delete_deidentify_template_with_org_id(self, get_conn): + self.hook.delete_deidentify_template( + template_id=TEMPLATE_ID, organization_id=ORGANIZATION_ID + ) + + get_conn.return_value.delete_deidentify_template.assert_called_once_with( + name=DEIDENTIFY_TEMPLATE_ORGANIZATION_PATH, + retry=None, + timeout=None, + metadata=None, + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_delete_deidentify_template_with_project_id(self, get_conn): + self.hook.delete_deidentify_template( + template_id=TEMPLATE_ID, project_id=PROJECT_ID + ) + + get_conn.return_value.delete_deidentify_template.assert_called_once_with( + name=DEIDENTIFY_TEMPLATE_PROJECT_PATH, + retry=None, + timeout=None, + metadata=None, + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_delete_deidentify_template_without_template_id(self, _): + with self.assertRaises(AirflowException): + self.hook.delete_deidentify_template(template_id=None) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_delete_deidentify_template_without_parent(self, _): + with self.assertRaises(AirflowException): + self.hook.delete_deidentify_template(template_id=TEMPLATE_ID) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_delete_dlp_job(self, get_conn): + self.hook.delete_dlp_job(dlp_job_id=DLP_JOB_ID, project_id=PROJECT_ID) + + get_conn.return_value.delete_dlp_job.assert_called_once_with( + name=DLP_JOB_PATH, retry=None, timeout=None, metadata=None + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_delete_dlp_job_without_dlp_job_id(self, _): + with self.assertRaises(AirflowException): + self.hook.delete_dlp_job(dlp_job_id=None, project_id=PROJECT_ID) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_delete_dlp_job_without_parent(self, _): + with self.assertRaises(AirflowException): + self.hook.delete_dlp_job(dlp_job_id=DLP_JOB_ID) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_delete_inspect_template_with_org_id(self, get_conn): + self.hook.delete_inspect_template( + template_id=TEMPLATE_ID, organization_id=ORGANIZATION_ID + ) + + get_conn.return_value.delete_inspect_template.assert_called_once_with( + name=INSPECT_TEMPLATE_ORGANIZATION_PATH, + retry=None, + timeout=None, + metadata=None, + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_delete_inspect_template_with_project_id(self, get_conn): + self.hook.delete_inspect_template( + template_id=TEMPLATE_ID, project_id=PROJECT_ID + ) + + get_conn.return_value.delete_inspect_template.assert_called_once_with( + name=INSPECT_TEMPLATE_PROJECT_PATH, + retry=None, + timeout=None, + metadata=None, + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_delete_inspect_template_without_template_id(self, _): + with self.assertRaises(AirflowException): + self.hook.delete_inspect_template(template_id=None) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_delete_inspect_template_without_parent(self, _): + with self.assertRaises(AirflowException): + self.hook.delete_inspect_template(template_id=TEMPLATE_ID) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_delete_job_trigger(self, get_conn): + self.hook.delete_job_trigger(job_trigger_id=TRIGGER_ID, project_id=PROJECT_ID) + + get_conn.return_value.delete_job_trigger.assert_called_once_with( + name=JOB_TRIGGER_PATH, retry=None, timeout=None, metadata=None + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_delete_job_trigger_without_trigger_id(self, _): + with self.assertRaises(AirflowException): + self.hook.delete_job_trigger(job_trigger_id=None, project_id=PROJECT_ID) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_delete_job_trigger_without_parent(self, _): + with self.assertRaises(AirflowException): + self.hook.delete_job_trigger(job_trigger_id=TRIGGER_ID) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_delete_stored_info_type_with_org_id(self, get_conn): + self.hook.delete_stored_info_type( + stored_info_type_id=STORED_INFO_TYPE_ID, organization_id=ORGANIZATION_ID + ) + + get_conn.return_value.delete_stored_info_type.assert_called_once_with( + name=STORED_INFO_TYPE_ORGANIZATION_PATH, + retry=None, + timeout=None, + metadata=None, + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_delete_stored_info_type_with_project_id(self, get_conn): + self.hook.delete_stored_info_type( + stored_info_type_id=STORED_INFO_TYPE_ID, project_id=PROJECT_ID + ) + + get_conn.return_value.delete_stored_info_type.assert_called_once_with( + name=STORED_INFO_TYPE_PROJECT_PATH, + retry=None, + timeout=None, + metadata=None, + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_delete_stored_info_type_without_stored_info_type_id(self, _): + with self.assertRaises(AirflowException): + self.hook.delete_stored_info_type(stored_info_type_id=None) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_delete_stored_info_type_without_parent(self, _): + with self.assertRaises(AirflowException): + self.hook.delete_stored_info_type(stored_info_type_id=STORED_INFO_TYPE_ID) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn", + **{ + "return_value.get_deidentify_template.return_value": API_RESPONSE + }, # type: ignore + ) + def test_get_deidentify_template_with_org_id(self, get_conn): + result = self.hook.get_deidentify_template( + template_id=TEMPLATE_ID, organization_id=ORGANIZATION_ID + ) + + self.assertIs(result, API_RESPONSE) + get_conn.return_value.get_deidentify_template.assert_called_once_with( + name=DEIDENTIFY_TEMPLATE_ORGANIZATION_PATH, + retry=None, + timeout=None, + metadata=None, + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn", + **{ + "return_value.get_deidentify_template.return_value": API_RESPONSE + }, # type: ignore + ) + def test_get_deidentify_template_with_project_id(self, get_conn): + result = self.hook.get_deidentify_template( + template_id=TEMPLATE_ID, project_id=PROJECT_ID + ) + + self.assertIs(result, API_RESPONSE) + get_conn.return_value.get_deidentify_template.assert_called_once_with( + name=DEIDENTIFY_TEMPLATE_PROJECT_PATH, + retry=None, + timeout=None, + metadata=None, + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_get_deidentify_template_without_template_id(self, _): + with self.assertRaises(AirflowException): + self.hook.get_deidentify_template(template_id=None) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_get_deidentify_template_without_parent(self, _): + with self.assertRaises(AirflowException): + self.hook.get_deidentify_template(template_id=TEMPLATE_ID) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn", + **{"return_value.get_dlp_job.return_value": API_RESPONSE}, # type: ignore + ) + def test_get_dlp_job(self, get_conn): + result = self.hook.get_dlp_job(dlp_job_id=DLP_JOB_ID, project_id=PROJECT_ID) + + self.assertIs(result, API_RESPONSE) + get_conn.return_value.get_dlp_job.assert_called_once_with( + name=DLP_JOB_PATH, retry=None, timeout=None, metadata=None + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_get_dlp_job_without_dlp_job_id(self, _): + with self.assertRaises(AirflowException): + self.hook.get_dlp_job(dlp_job_id=None, project_id=PROJECT_ID) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_get_dlp_job_without_parent(self, _): + with self.assertRaises(AirflowException): + self.hook.get_dlp_job(dlp_job_id=DLP_JOB_ID) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn", + **{ + "return_value.get_inspect_template.return_value": API_RESPONSE + }, # type: ignore + ) + def test_get_inspect_template_with_org_id(self, get_conn): + result = self.hook.get_inspect_template( + template_id=TEMPLATE_ID, organization_id=ORGANIZATION_ID + ) + + self.assertIs(result, API_RESPONSE) + get_conn.return_value.get_inspect_template.assert_called_once_with( + name=INSPECT_TEMPLATE_ORGANIZATION_PATH, + retry=None, + timeout=None, + metadata=None, + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn", + **{ + "return_value.get_inspect_template.return_value": API_RESPONSE + }, # type: ignore + ) + def test_get_inspect_template_with_project_id(self, get_conn): + result = self.hook.get_inspect_template( + template_id=TEMPLATE_ID, project_id=PROJECT_ID + ) + + self.assertIs(result, API_RESPONSE) + get_conn.return_value.get_inspect_template.assert_called_once_with( + name=INSPECT_TEMPLATE_PROJECT_PATH, + retry=None, + timeout=None, + metadata=None, + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_get_inspect_template_without_template_id(self, _): + with self.assertRaises(AirflowException): + self.hook.get_inspect_template(template_id=None) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_get_inspect_template_without_parent(self, _): + with self.assertRaises(AirflowException): + self.hook.get_inspect_template(template_id=TEMPLATE_ID) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn", + **{"return_value.get_job_trigger.return_value": API_RESPONSE}, # type: ignore + ) + def test_get_job_trigger(self, get_conn): + result = self.hook.get_job_trigger( + job_trigger_id=TRIGGER_ID, project_id=PROJECT_ID + ) + + self.assertIs(result, API_RESPONSE) + get_conn.return_value.get_job_trigger.assert_called_once_with( + name=JOB_TRIGGER_PATH, retry=None, timeout=None, metadata=None + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_get_job_trigger_without_trigger_id(self, _): + with self.assertRaises(AirflowException): + self.hook.get_job_trigger(job_trigger_id=None, project_id=PROJECT_ID) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_get_job_trigger_without_parent(self, _): + with self.assertRaises(AirflowException): + self.hook.get_job_trigger(job_trigger_id=TRIGGER_ID) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn", + **{ + "return_value.get_stored_info_type.return_value": API_RESPONSE + }, # type: ignore + ) + def test_get_stored_info_type_with_org_id(self, get_conn): + result = self.hook.get_stored_info_type( + stored_info_type_id=STORED_INFO_TYPE_ID, organization_id=ORGANIZATION_ID + ) + + self.assertIs(result, API_RESPONSE) + get_conn.return_value.get_stored_info_type.assert_called_once_with( + name=STORED_INFO_TYPE_ORGANIZATION_PATH, + retry=None, + timeout=None, + metadata=None, + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn", + **{ + "return_value.get_stored_info_type.return_value": API_RESPONSE + }, # type: ignore + ) + def test_get_stored_info_type_with_project_id(self, get_conn): + result = self.hook.get_stored_info_type( + stored_info_type_id=STORED_INFO_TYPE_ID, project_id=PROJECT_ID + ) + + self.assertIs(result, API_RESPONSE) + get_conn.return_value.get_stored_info_type.assert_called_once_with( + name=STORED_INFO_TYPE_PROJECT_PATH, + retry=None, + timeout=None, + metadata=None, + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_get_stored_info_type_without_stored_info_type_id(self, _): + with self.assertRaises(AirflowException): + self.hook.get_stored_info_type(stored_info_type_id=None) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_get_stored_info_type_without_parent(self, _): + with self.assertRaises(AirflowException): + self.hook.get_stored_info_type(stored_info_type_id=STORED_INFO_TYPE_ID) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn", + **{"return_value.inspect_content.return_value": API_RESPONSE}, # type: ignore + ) + def test_inspect_content(self, get_conn): + result = self.hook.inspect_content(project_id=PROJECT_ID) + + self.assertIs(result, API_RESPONSE) + get_conn.return_value.inspect_content.assert_called_once_with( + parent=PROJECT_PATH, + inspect_config=None, + item=None, + inspect_template_name=None, + retry=None, + timeout=None, + metadata=None, + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_inspect_content_without_parent(self, _): + with self.assertRaises(AirflowException): + self.hook.inspect_content() + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_list_deidentify_templates_with_org_id(self, get_conn): + result = self.hook.list_deidentify_templates(organization_id=ORGANIZATION_ID) + + self.assertIsInstance(result, list) + get_conn.return_value.list_deidentify_templates.assert_called_once_with( + parent=ORGANIZATION_PATH, + page_size=None, + order_by=None, + retry=None, + timeout=None, + metadata=None, + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_list_deidentify_templates_with_project_id(self, get_conn): + result = self.hook.list_deidentify_templates(project_id=PROJECT_ID) + + self.assertIsInstance(result, list) + get_conn.return_value.list_deidentify_templates.assert_called_once_with( + parent=PROJECT_PATH, + page_size=None, + order_by=None, + retry=None, + timeout=None, + metadata=None, + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_list_deidentify_templates_without_parent(self, _): + with self.assertRaises(AirflowException): + self.hook.list_deidentify_templates() + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_list_dlp_jobs(self, get_conn): + result = self.hook.list_dlp_jobs(project_id=PROJECT_ID) + + self.assertIsInstance(result, list) + get_conn.return_value.list_dlp_jobs.assert_called_once_with( + parent=PROJECT_PATH, + filter_=None, + page_size=None, + type_=None, + order_by=None, + retry=None, + timeout=None, + metadata=None, + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_list_dlp_jobs_without_parent(self, _): + with self.assertRaises(AirflowException): + self.hook.list_dlp_jobs() + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn", + **{"return_value.list_info_types.return_value": API_RESPONSE}, # type: ignore + ) + def test_list_info_types(self, get_conn): + result = self.hook.list_info_types() + + self.assertIs(result, API_RESPONSE) + get_conn.return_value.list_info_types.assert_called_once_with( + language_code=None, filter_=None, retry=None, timeout=None, metadata=None + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_list_inspect_templates_with_org_id(self, get_conn): + result = self.hook.list_inspect_templates(organization_id=ORGANIZATION_ID) + + self.assertIsInstance(result, list) + get_conn.return_value.list_inspect_templates.assert_called_once_with( + parent=ORGANIZATION_PATH, + page_size=None, + order_by=None, + retry=None, + timeout=None, + metadata=None, + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_list_inspect_templates_with_project_id(self, get_conn): + result = self.hook.list_inspect_templates(project_id=PROJECT_ID) + + self.assertIsInstance(result, list) + get_conn.return_value.list_inspect_templates.assert_called_once_with( + parent=PROJECT_PATH, + page_size=None, + order_by=None, + retry=None, + timeout=None, + metadata=None, + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_list_inspect_templates_without_parent(self, _): + with self.assertRaises(AirflowException): + self.hook.list_inspect_templates() + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_list_job_triggers(self, get_conn): + result = self.hook.list_job_triggers(project_id=PROJECT_ID) + + self.assertIsInstance(result, list) + get_conn.return_value.list_job_triggers.assert_called_once_with( + parent=PROJECT_PATH, + page_size=None, + order_by=None, + filter_=None, + retry=None, + timeout=None, + metadata=None, + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_list_job_triggers_without_parent(self, _): + with self.assertRaises(AirflowException): + self.hook.list_job_triggers() + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_list_stored_info_types_with_org_id(self, get_conn): + result = self.hook.list_stored_info_types(organization_id=ORGANIZATION_ID) + + self.assertIsInstance(result, list) + get_conn.return_value.list_stored_info_types.assert_called_once_with( + parent=ORGANIZATION_PATH, + page_size=None, + order_by=None, + retry=None, + timeout=None, + metadata=None, + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_list_stored_info_types_with_project_id(self, get_conn): + result = self.hook.list_stored_info_types(project_id=PROJECT_ID) + + self.assertIsInstance(result, list) + get_conn.return_value.list_stored_info_types.assert_called_once_with( + parent=PROJECT_PATH, + page_size=None, + order_by=None, + retry=None, + timeout=None, + metadata=None, + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_list_stored_info_types_without_parent(self, _): + with self.assertRaises(AirflowException): + self.hook.list_stored_info_types() + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn", + **{"return_value.redact_image.return_value": API_RESPONSE}, # type: ignore + ) + def test_redact_image(self, get_conn): + result = self.hook.redact_image(project_id=PROJECT_ID) + + self.assertIs(result, API_RESPONSE) + get_conn.return_value.redact_image.assert_called_once_with( + parent=PROJECT_PATH, + inspect_config=None, + image_redaction_configs=None, + include_findings=None, + byte_item=None, + retry=None, + timeout=None, + metadata=None, + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_redact_image_without_parent(self, _): + with self.assertRaises(AirflowException): + self.hook.redact_image() + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn", + **{ + "return_value.reidentify_content.return_value": API_RESPONSE + }, # type: ignore + ) + def test_reidentify_content(self, get_conn): + result = self.hook.reidentify_content(project_id=PROJECT_ID) + + self.assertIs(result, API_RESPONSE) + get_conn.return_value.reidentify_content.assert_called_once_with( + parent=PROJECT_PATH, + reidentify_config=None, + inspect_config=None, + item=None, + inspect_template_name=None, + reidentify_template_name=None, + retry=None, + timeout=None, + metadata=None, + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_reidentify_content_without_parent(self, _): + with self.assertRaises(AirflowException): + self.hook.reidentify_content() + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn", + **{ + "return_value.update_deidentify_template.return_value": API_RESPONSE + }, # type: ignore + ) + def test_update_deidentify_template_with_org_id(self, get_conn): + result = self.hook.update_deidentify_template( + template_id=TEMPLATE_ID, organization_id=ORGANIZATION_ID + ) + + self.assertIs(result, API_RESPONSE) + get_conn.return_value.update_deidentify_template.assert_called_once_with( + name=DEIDENTIFY_TEMPLATE_ORGANIZATION_PATH, + deidentify_template=None, + update_mask=None, + retry=None, + timeout=None, + metadata=None, + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn", + **{ + "return_value.update_deidentify_template.return_value": API_RESPONSE + }, # type: ignore + ) + def test_update_deidentify_template_with_project_id(self, get_conn): + result = self.hook.update_deidentify_template( + template_id=TEMPLATE_ID, project_id=PROJECT_ID + ) + + self.assertIs(result, API_RESPONSE) + get_conn.return_value.update_deidentify_template.assert_called_once_with( + name=DEIDENTIFY_TEMPLATE_PROJECT_PATH, + deidentify_template=None, + update_mask=None, + retry=None, + timeout=None, + metadata=None, + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_update_deidentify_template_without_template_id(self, _): + with self.assertRaises(AirflowException): + self.hook.update_deidentify_template( + template_id=None, organization_id=ORGANIZATION_ID + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_update_deidentify_template_without_parent(self, _): + with self.assertRaises(AirflowException): + self.hook.update_deidentify_template(template_id=TEMPLATE_ID) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn", + **{ + "return_value.update_inspect_template.return_value": API_RESPONSE + }, # type: ignore + ) + def test_update_inspect_template_with_org_id(self, get_conn): + result = self.hook.update_inspect_template( + template_id=TEMPLATE_ID, organization_id=ORGANIZATION_ID + ) + + self.assertIs(result, API_RESPONSE) + get_conn.return_value.update_inspect_template.assert_called_once_with( + name=INSPECT_TEMPLATE_ORGANIZATION_PATH, + inspect_template=None, + update_mask=None, + retry=None, + timeout=None, + metadata=None, + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn", + **{ + "return_value.update_inspect_template.return_value": API_RESPONSE + }, # type: ignore + ) + def test_update_inspect_template_with_project_id(self, get_conn): + result = self.hook.update_inspect_template( + template_id=TEMPLATE_ID, project_id=PROJECT_ID + ) + + self.assertIs(result, API_RESPONSE) + get_conn.return_value.update_inspect_template.assert_called_once_with( + name=INSPECT_TEMPLATE_PROJECT_PATH, + inspect_template=None, + update_mask=None, + retry=None, + timeout=None, + metadata=None, + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_update_inspect_template_without_template_id(self, _): + with self.assertRaises(AirflowException): + self.hook.update_inspect_template( + template_id=None, organization_id=ORGANIZATION_ID + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_update_inspect_template_without_parent(self, _): + with self.assertRaises(AirflowException): + self.hook.update_inspect_template(template_id=TEMPLATE_ID) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn", + **{ + "return_value.update_job_trigger.return_value": API_RESPONSE + }, # type: ignore + ) + def test_update_job_trigger(self, get_conn): + result = self.hook.update_job_trigger( + job_trigger_id=TRIGGER_ID, project_id=PROJECT_ID + ) + + self.assertIs(result, API_RESPONSE) + get_conn.return_value.update_job_trigger.assert_called_once_with( + name=JOB_TRIGGER_PATH, + job_trigger=None, + update_mask=None, + retry=None, + timeout=None, + metadata=None, + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_update_job_trigger_without_job_trigger_id(self, _): + with self.assertRaises(AirflowException): + self.hook.update_job_trigger(job_trigger_id=None, project_id=PROJECT_ID) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_update_job_trigger_without_parent(self, _): + with self.assertRaises(AirflowException): + self.hook.update_job_trigger(job_trigger_id=TRIGGER_ID) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn", + **{ + "return_value.update_stored_info_type.return_value": API_RESPONSE + }, # type: ignore + ) + def test_update_stored_info_type_with_org_id(self, get_conn): + result = self.hook.update_stored_info_type( + stored_info_type_id=STORED_INFO_TYPE_ID, organization_id=ORGANIZATION_ID + ) + + self.assertIs(result, API_RESPONSE) + get_conn.return_value.update_stored_info_type.assert_called_once_with( + name=STORED_INFO_TYPE_ORGANIZATION_PATH, + config=None, + update_mask=None, + retry=None, + timeout=None, + metadata=None, + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn", + **{ + "return_value.update_stored_info_type.return_value": API_RESPONSE + }, # type: ignore + ) + def test_update_stored_info_type_with_project_id(self, get_conn): + result = self.hook.update_stored_info_type( + stored_info_type_id=STORED_INFO_TYPE_ID, project_id=PROJECT_ID + ) + + self.assertIs(result, API_RESPONSE) + get_conn.return_value.update_stored_info_type.assert_called_once_with( + name=STORED_INFO_TYPE_PROJECT_PATH, + config=None, + update_mask=None, + retry=None, + timeout=None, + metadata=None, + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_update_stored_info_type_without_stored_info_type_id(self, _): + with self.assertRaises(AirflowException): + self.hook.update_stored_info_type( + stored_info_type_id=None, organization_id=ORGANIZATION_ID + ) + + @mock.patch( # type: ignore + "airflow.contrib.hooks.gcp_dlp_hook.CloudDLPHook.get_conn" + ) + def test_update_stored_info_type_without_parent(self, _): + with self.assertRaises(AirflowException): + self.hook.update_stored_info_type(stored_info_type_id=STORED_INFO_TYPE_ID) diff --git a/tests/contrib/operators/test_gcp_dlp_operator.py b/tests/contrib/operators/test_gcp_dlp_operator.py new file mode 100644 index 0000000000000..5a92d5ae0beb4 --- /dev/null +++ b/tests/contrib/operators/test_gcp_dlp_operator.py @@ -0,0 +1,648 @@ +# -*- coding: utf-8 -*- +# +# 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. + +# pylint: disable=R0904, C0111 +""" +This module contains various unit tests for GCP DLP Operators +""" + +import unittest + +from airflow.contrib.operators.gcp_dlp_operator import ( + CloudDLPCancelDLPJobOperator, + CloudDLPCreateDeidentifyTemplateOperator, + CloudDLPCreateDLPJobOperator, + CloudDLPCreateInspectTemplateOperator, + CloudDLPCreateJobTriggerOperator, + CloudDLPCreateStoredInfoTypeOperator, + CloudDLPDeidentifyContentOperator, + CloudDLPDeleteDeidentifyTemplateOperator, + CloudDLPDeleteInspectTemplateOperator, + CloudDLPDeleteDlpJobOperator, + CloudDLPDeleteJobTriggerOperator, + CloudDLPDeleteStoredInfoTypeOperator, + CloudDLPGetDeidentifyTemplateOperator, + CloudDLPGetDlpJobOperator, + CloudDLPGetInspectTemplateOperator, + CloudDLPGetJobTripperOperator, + CloudDLPGetStoredInfoTypeOperator, + CloudDLPInspectContentOperator, + CloudDLPListDeidentifyTemplatesOperator, + CloudDLPListDlpJobsOperator, + CloudDLPListInfoTypesOperator, + CloudDLPListInspectTemplatesOperator, + CloudDLPListJobTriggersOperator, + CloudDLPListStoredInfoTypesOperator, + CloudDLPRedactImageOperator, + CloudDLPReidentifyContentOperator, + CloudDLPUpdateDeidentifyTemplateOperator, + CloudDLPUpdateInspectTemplateOperator, + CloudDLPUpdateJobTriggerOperator, + CloudDLPUpdateStoredInfoTypeOperator, +) +from tests.compat import mock + +GCP_CONN_ID = "google_cloud_default" +ORGANIZATION_ID = "test-org" +PROJECT_ID = "test-project" +DLP_JOB_ID = "job123" +TEMPLATE_ID = "template123" +STORED_INFO_TYPE_ID = "type123" +TRIGGER_ID = "trigger123" + + +class CloudDLPCancelDLPJobOperatorTest(unittest.TestCase): + @mock.patch("airflow.contrib.operators.gcp_dlp_operator.CloudDLPHook") + def test_cancel_dlp_job(self, mock_hook): + mock_hook.return_value.cancel_dlp_job.return_value = {} + operator = CloudDLPCancelDLPJobOperator(dlp_job_id=DLP_JOB_ID, task_id="id") + operator.execute(context=None) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID) + mock_hook.return_value.cancel_dlp_job.assert_called_once_with( + dlp_job_id=DLP_JOB_ID, + project_id=None, + retry=None, + timeout=None, + metadata=None, + ) + + +class CloudDLPCreateDeidentifyTemplateOperatorTest(unittest.TestCase): + @mock.patch("airflow.contrib.operators.gcp_dlp_operator.CloudDLPHook") + def test_create_deidentify_template(self, mock_hook): + mock_hook.return_value.create_deidentify_template.return_value = {} + operator = CloudDLPCreateDeidentifyTemplateOperator( + organization_id=ORGANIZATION_ID, task_id="id" + ) + operator.execute(context=None) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID) + mock_hook.return_value.create_deidentify_template.assert_called_once_with( + organization_id=ORGANIZATION_ID, + project_id=None, + deidentify_template=None, + template_id=None, + retry=None, + timeout=None, + metadata=None, + ) + + +class CloudDLPCreateDLPJobOperatorTest(unittest.TestCase): + @mock.patch("airflow.contrib.operators.gcp_dlp_operator.CloudDLPHook") + def test_create_dlp_job(self, mock_hook): + mock_hook.return_value.create_dlp_job.return_value = {} + operator = CloudDLPCreateDLPJobOperator(project_id=PROJECT_ID, task_id="id") + operator.execute(context=None) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID) + mock_hook.return_value.create_dlp_job.assert_called_once_with( + project_id=PROJECT_ID, + inspect_job=None, + risk_job=None, + job_id=None, + retry=None, + timeout=None, + metadata=None, + wait_until_finished=True, + ) + + +class CloudDLPCreateInspectTemplateOperatorTest(unittest.TestCase): + @mock.patch("airflow.contrib.operators.gcp_dlp_operator.CloudDLPHook") + def test_create_inspect_template(self, mock_hook): + mock_hook.return_value.create_inspect_template.return_value = {} + operator = CloudDLPCreateInspectTemplateOperator( + organization_id=ORGANIZATION_ID, task_id="id" + ) + operator.execute(context=None) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID) + mock_hook.return_value.create_inspect_template.assert_called_once_with( + organization_id=ORGANIZATION_ID, + project_id=None, + inspect_template=None, + template_id=None, + retry=None, + timeout=None, + metadata=None, + ) + + +class CloudDLPCreateJobTriggerOperatorTest(unittest.TestCase): + @mock.patch("airflow.contrib.operators.gcp_dlp_operator.CloudDLPHook") + def test_create_job_trigger(self, mock_hook): + mock_hook.return_value.create_job_trigger.return_value = {} + operator = CloudDLPCreateJobTriggerOperator(project_id=PROJECT_ID, task_id="id") + operator.execute(context=None) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID) + mock_hook.return_value.create_job_trigger.assert_called_once_with( + project_id=PROJECT_ID, + job_trigger=None, + trigger_id=None, + retry=None, + timeout=None, + metadata=None, + ) + + +class CloudDLPCreateStoredInfoTypeOperatorTest(unittest.TestCase): + @mock.patch("airflow.contrib.operators.gcp_dlp_operator.CloudDLPHook") + def test_create_stored_info_type(self, mock_hook): + mock_hook.return_value.create_stored_info_type.return_value = {} + operator = CloudDLPCreateStoredInfoTypeOperator( + organization_id=ORGANIZATION_ID, task_id="id" + ) + operator.execute(context=None) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID) + mock_hook.return_value.create_stored_info_type.assert_called_once_with( + organization_id=ORGANIZATION_ID, + project_id=None, + config=None, + stored_info_type_id=None, + retry=None, + timeout=None, + metadata=None, + ) + + +class CloudDLPDeidentifyContentOperatorTest(unittest.TestCase): + @mock.patch("airflow.contrib.operators.gcp_dlp_operator.CloudDLPHook") + def test_deidentify_content(self, mock_hook): + mock_hook.return_value.deidentify_content.return_value = {} + operator = CloudDLPDeidentifyContentOperator( + project_id=PROJECT_ID, task_id="id" + ) + operator.execute(context=None) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID) + mock_hook.return_value.deidentify_content.assert_called_once_with( + project_id=PROJECT_ID, + deidentify_config=None, + inspect_config=None, + item=None, + inspect_template_name=None, + deidentify_template_name=None, + retry=None, + timeout=None, + metadata=None, + ) + + +class CloudDLPDeleteDeidentifyTemplateOperatorTest(unittest.TestCase): + @mock.patch("airflow.contrib.operators.gcp_dlp_operator.CloudDLPHook") + def test_delete_deidentify_template(self, mock_hook): + mock_hook.return_value.delete_deidentify_template.return_value = {} + operator = CloudDLPDeleteDeidentifyTemplateOperator( + template_id=TEMPLATE_ID, organization_id=ORGANIZATION_ID, task_id="id" + ) + operator.execute(context=None) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID) + mock_hook.return_value.delete_deidentify_template.assert_called_once_with( + template_id=TEMPLATE_ID, + organization_id=ORGANIZATION_ID, + project_id=None, + retry=None, + timeout=None, + metadata=None, + ) + + +class CloudDLPDeleteDlpJobOperatorTest(unittest.TestCase): + @mock.patch("airflow.contrib.operators.gcp_dlp_operator.CloudDLPHook") + def test_delete_dlp_job(self, mock_hook): + mock_hook.return_value.delete_dlp_job.return_value = {} + operator = CloudDLPDeleteDlpJobOperator( + dlp_job_id=DLP_JOB_ID, project_id=PROJECT_ID, task_id="id" + ) + operator.execute(context=None) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID) + mock_hook.return_value.delete_dlp_job.assert_called_once_with( + dlp_job_id=DLP_JOB_ID, + project_id=PROJECT_ID, + retry=None, + timeout=None, + metadata=None, + ) + + +class CloudDLPDeleteInspectTemplateOperatorTest(unittest.TestCase): + @mock.patch("airflow.contrib.operators.gcp_dlp_operator.CloudDLPHook") + def test_delete_inspect_template(self, mock_hook): + mock_hook.return_value.delete_inspect_template.return_value = {} + operator = CloudDLPDeleteInspectTemplateOperator( + template_id=TEMPLATE_ID, organization_id=ORGANIZATION_ID, task_id="id" + ) + operator.execute(context=None) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID) + mock_hook.return_value.delete_inspect_template.assert_called_once_with( + template_id=TEMPLATE_ID, + organization_id=ORGANIZATION_ID, + project_id=None, + retry=None, + timeout=None, + metadata=None, + ) + + +class CloudDLPDeleteJobTriggerOperatorTest(unittest.TestCase): + @mock.patch("airflow.contrib.operators.gcp_dlp_operator.CloudDLPHook") + def test_delete_job_trigger(self, mock_hook): + mock_hook.return_value.delete_job_trigger.return_value = {} + operator = CloudDLPDeleteJobTriggerOperator( + job_trigger_id=TRIGGER_ID, project_id=PROJECT_ID, task_id="id" + ) + operator.execute(context=None) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID) + mock_hook.return_value.delete_job_trigger.assert_called_once_with( + job_trigger_id=TRIGGER_ID, + project_id=PROJECT_ID, + retry=None, + timeout=None, + metadata=None, + ) + + +class CloudDLPDeleteStoredInfoTypeOperatorTest(unittest.TestCase): + @mock.patch("airflow.contrib.operators.gcp_dlp_operator.CloudDLPHook") + def test_delete_stored_info_type(self, mock_hook): + mock_hook.return_value.delete_stored_info_type.return_value = {} + operator = CloudDLPDeleteStoredInfoTypeOperator( + stored_info_type_id=STORED_INFO_TYPE_ID, + organization_id=ORGANIZATION_ID, + task_id="id", + ) + operator.execute(context=None) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID) + mock_hook.return_value.delete_stored_info_type.assert_called_once_with( + stored_info_type_id=STORED_INFO_TYPE_ID, + organization_id=ORGANIZATION_ID, + project_id=None, + retry=None, + timeout=None, + metadata=None, + ) + + +class CloudDLPGetDeidentifyTemplateOperatorTest(unittest.TestCase): + @mock.patch("airflow.contrib.operators.gcp_dlp_operator.CloudDLPHook") + def test_get_deidentify_template(self, mock_hook): + mock_hook.return_value.get_deidentify_template.return_value = {} + operator = CloudDLPGetDeidentifyTemplateOperator( + template_id=TEMPLATE_ID, organization_id=ORGANIZATION_ID, task_id="id" + ) + operator.execute(context=None) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID) + mock_hook.return_value.get_deidentify_template.assert_called_once_with( + template_id=TEMPLATE_ID, + organization_id=ORGANIZATION_ID, + project_id=None, + retry=None, + timeout=None, + metadata=None, + ) + + +class CloudDLPGetDlpJobOperatorTest(unittest.TestCase): + @mock.patch("airflow.contrib.operators.gcp_dlp_operator.CloudDLPHook") + def test_get_dlp_job(self, mock_hook): + mock_hook.return_value.get_dlp_job.return_value = {} + operator = CloudDLPGetDlpJobOperator( + dlp_job_id=DLP_JOB_ID, project_id=PROJECT_ID, task_id="id" + ) + operator.execute(context=None) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID) + mock_hook.return_value.get_dlp_job.assert_called_once_with( + dlp_job_id=DLP_JOB_ID, + project_id=PROJECT_ID, + retry=None, + timeout=None, + metadata=None, + ) + + +class CloudDLPGetInspectTemplateOperatorTest(unittest.TestCase): + @mock.patch("airflow.contrib.operators.gcp_dlp_operator.CloudDLPHook") + def test_get_inspect_template(self, mock_hook): + mock_hook.return_value.get_inspect_template.return_value = {} + operator = CloudDLPGetInspectTemplateOperator( + template_id=TEMPLATE_ID, organization_id=ORGANIZATION_ID, task_id="id" + ) + operator.execute(context=None) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID) + mock_hook.return_value.get_inspect_template.assert_called_once_with( + template_id=TEMPLATE_ID, + organization_id=ORGANIZATION_ID, + project_id=None, + retry=None, + timeout=None, + metadata=None, + ) + + +class CloudDLPGetJobTripperOperatorTest(unittest.TestCase): + @mock.patch("airflow.contrib.operators.gcp_dlp_operator.CloudDLPHook") + def test_get_job_trigger(self, mock_hook): + mock_hook.return_value.get_job_trigger.return_value = {} + operator = CloudDLPGetJobTripperOperator( + job_trigger_id=TRIGGER_ID, project_id=PROJECT_ID, task_id="id" + ) + operator.execute(context=None) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID) + mock_hook.return_value.get_job_trigger.assert_called_once_with( + job_trigger_id=TRIGGER_ID, + project_id=PROJECT_ID, + retry=None, + timeout=None, + metadata=None, + ) + + +class CloudDLPGetStoredInfoTypeOperatorTest(unittest.TestCase): + @mock.patch("airflow.contrib.operators.gcp_dlp_operator.CloudDLPHook") + def test_get_stored_info_type(self, mock_hook): + mock_hook.return_value.get_stored_info_type.return_value = {} + operator = CloudDLPGetStoredInfoTypeOperator( + stored_info_type_id=STORED_INFO_TYPE_ID, + organization_id=ORGANIZATION_ID, + task_id="id", + ) + operator.execute(context=None) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID) + mock_hook.return_value.get_stored_info_type.assert_called_once_with( + stored_info_type_id=STORED_INFO_TYPE_ID, + organization_id=ORGANIZATION_ID, + project_id=None, + retry=None, + timeout=None, + metadata=None, + ) + + +class CloudDLPInspectContentOperatorTest(unittest.TestCase): + @mock.patch("airflow.contrib.operators.gcp_dlp_operator.CloudDLPHook") + def test_inspect_content(self, mock_hook): + mock_hook.return_value.inspect_content.return_value = {} + operator = CloudDLPInspectContentOperator(project_id=PROJECT_ID, task_id="id") + operator.execute(context=None) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID) + mock_hook.return_value.inspect_content.assert_called_once_with( + project_id=PROJECT_ID, + inspect_config=None, + item=None, + inspect_template_name=None, + retry=None, + timeout=None, + metadata=None, + ) + + +class CloudDLPListDeidentifyTemplatesOperatorTest(unittest.TestCase): + @mock.patch("airflow.contrib.operators.gcp_dlp_operator.CloudDLPHook") + def test_list_deidentify_templates(self, mock_hook): + mock_hook.return_value.list_deidentify_templates.return_value = {} + operator = CloudDLPListDeidentifyTemplatesOperator( + organization_id=ORGANIZATION_ID, task_id="id" + ) + operator.execute(context=None) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID) + mock_hook.return_value.list_deidentify_templates.assert_called_once_with( + organization_id=ORGANIZATION_ID, + project_id=None, + page_size=None, + order_by=None, + retry=None, + timeout=None, + metadata=None, + ) + + +class CloudDLPListDlpJobsOperatorTest(unittest.TestCase): + @mock.patch("airflow.contrib.operators.gcp_dlp_operator.CloudDLPHook") + def test_list_dlp_jobs(self, mock_hook): + mock_hook.return_value.list_dlp_jobs.return_value = {} + operator = CloudDLPListDlpJobsOperator(project_id=PROJECT_ID, task_id="id") + operator.execute(context=None) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID) + mock_hook.return_value.list_dlp_jobs.assert_called_once_with( + project_id=PROJECT_ID, + results_filter=None, + page_size=None, + job_type=None, + order_by=None, + retry=None, + timeout=None, + metadata=None, + ) + + +class CloudDLPListInfoTypesOperatorTest(unittest.TestCase): + @mock.patch("airflow.contrib.operators.gcp_dlp_operator.CloudDLPHook") + def test_list_info_types(self, mock_hook): + mock_hook.return_value.list_info_types.return_value = {} + operator = CloudDLPListInfoTypesOperator(task_id="id") + operator.execute(context=None) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID) + mock_hook.return_value.list_info_types.assert_called_once_with( + language_code=None, + results_filter=None, + retry=None, + timeout=None, + metadata=None, + ) + + +class CloudDLPListInspectTemplatesOperatorTest(unittest.TestCase): + @mock.patch("airflow.contrib.operators.gcp_dlp_operator.CloudDLPHook") + def test_list_inspect_templates(self, mock_hook): + mock_hook.return_value.list_inspect_templates.return_value = {} + operator = CloudDLPListInspectTemplatesOperator( + organization_id=ORGANIZATION_ID, task_id="id" + ) + operator.execute(context=None) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID) + mock_hook.return_value.list_inspect_templates.assert_called_once_with( + organization_id=ORGANIZATION_ID, + project_id=None, + page_size=None, + order_by=None, + retry=None, + timeout=None, + metadata=None, + ) + + +class CloudDLPListJobTriggersOperatorTest(unittest.TestCase): + @mock.patch("airflow.contrib.operators.gcp_dlp_operator.CloudDLPHook") + def test_list_job_triggers(self, mock_hook): + mock_hook.return_value.list_job_triggers.return_value = {} + operator = CloudDLPListJobTriggersOperator(project_id=PROJECT_ID, task_id="id") + operator.execute(context=None) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID) + mock_hook.return_value.list_job_triggers.assert_called_once_with( + project_id=PROJECT_ID, + page_size=None, + order_by=None, + results_filter=None, + retry=None, + timeout=None, + metadata=None, + ) + + +class CloudDLPListStoredInfoTypesOperatorTest(unittest.TestCase): + @mock.patch("airflow.contrib.operators.gcp_dlp_operator.CloudDLPHook") + def test_list_stored_info_types(self, mock_hook): + mock_hook.return_value.list_stored_info_types.return_value = {} + operator = CloudDLPListStoredInfoTypesOperator( + organization_id=ORGANIZATION_ID, task_id="id" + ) + operator.execute(context=None) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID) + mock_hook.return_value.list_stored_info_types.assert_called_once_with( + organization_id=ORGANIZATION_ID, + project_id=None, + page_size=None, + order_by=None, + retry=None, + timeout=None, + metadata=None, + ) + + +class CloudDLPRedactImageOperatorTest(unittest.TestCase): + @mock.patch("airflow.contrib.operators.gcp_dlp_operator.CloudDLPHook") + def test_redact_image(self, mock_hook): + mock_hook.return_value.redact_image.return_value = {} + operator = CloudDLPRedactImageOperator(project_id=PROJECT_ID, task_id="id") + operator.execute(context=None) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID) + mock_hook.return_value.redact_image.assert_called_once_with( + project_id=PROJECT_ID, + inspect_config=None, + image_redaction_configs=None, + include_findings=None, + byte_item=None, + retry=None, + timeout=None, + metadata=None, + ) + + +class CloudDLPReidentifyContentOperatorTest(unittest.TestCase): + @mock.patch("airflow.contrib.operators.gcp_dlp_operator.CloudDLPHook") + def test_reidentify_content(self, mock_hook): + mock_hook.return_value.reidentify_content.return_value = {} + operator = CloudDLPReidentifyContentOperator( + project_id=PROJECT_ID, task_id="id" + ) + operator.execute(context=None) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID) + mock_hook.return_value.reidentify_content.assert_called_once_with( + project_id=PROJECT_ID, + reidentify_config=None, + inspect_config=None, + item=None, + inspect_template_name=None, + reidentify_template_name=None, + retry=None, + timeout=None, + metadata=None, + ) + + +class CloudDLPUpdateDeidentifyTemplateOperatorTest(unittest.TestCase): + @mock.patch("airflow.contrib.operators.gcp_dlp_operator.CloudDLPHook") + def test_update_deidentify_template(self, mock_hook): + mock_hook.return_value.update_deidentify_template.return_value = {} + operator = CloudDLPUpdateDeidentifyTemplateOperator( + template_id=TEMPLATE_ID, organization_id=ORGANIZATION_ID, task_id="id" + ) + operator.execute(context=None) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID) + mock_hook.return_value.update_deidentify_template.assert_called_once_with( + template_id=TEMPLATE_ID, + organization_id=ORGANIZATION_ID, + project_id=None, + deidentify_template=None, + update_mask=None, + retry=None, + timeout=None, + metadata=None, + ) + + +class CloudDLPUpdateInspectTemplateOperatorTest(unittest.TestCase): + @mock.patch("airflow.contrib.operators.gcp_dlp_operator.CloudDLPHook") + def test_update_inspect_template(self, mock_hook): + mock_hook.return_value.update_inspect_template.return_value = {} + operator = CloudDLPUpdateInspectTemplateOperator( + template_id=TEMPLATE_ID, organization_id=ORGANIZATION_ID, task_id="id" + ) + operator.execute(context=None) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID) + mock_hook.return_value.update_inspect_template.assert_called_once_with( + template_id=TEMPLATE_ID, + organization_id=ORGANIZATION_ID, + project_id=None, + inspect_template=None, + update_mask=None, + retry=None, + timeout=None, + metadata=None, + ) + + +class CloudDLPUpdateJobTriggerOperatorTest(unittest.TestCase): + @mock.patch("airflow.contrib.operators.gcp_dlp_operator.CloudDLPHook") + def test_update_job_trigger(self, mock_hook): + mock_hook.return_value.update_job_trigger.return_value = {} + operator = CloudDLPUpdateJobTriggerOperator( + job_trigger_id=TRIGGER_ID, task_id="id" + ) + operator.execute(context=None) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID) + mock_hook.return_value.update_job_trigger.assert_called_once_with( + job_trigger_id=TRIGGER_ID, + project_id=None, + job_trigger=None, + update_mask=None, + retry=None, + timeout=None, + metadata=None, + ) + + +class CloudDLPUpdateStoredInfoTypeOperatorTest(unittest.TestCase): + @mock.patch("airflow.contrib.operators.gcp_dlp_operator.CloudDLPHook") + def test_update_stored_info_type(self, mock_hook): + mock_hook.return_value.update_stored_info_type.return_value = {} + operator = CloudDLPUpdateStoredInfoTypeOperator( + stored_info_type_id=STORED_INFO_TYPE_ID, + organization_id=ORGANIZATION_ID, + task_id="id", + ) + operator.execute(context=None) + mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID) + mock_hook.return_value.update_stored_info_type.assert_called_once_with( + stored_info_type_id=STORED_INFO_TYPE_ID, + organization_id=ORGANIZATION_ID, + project_id=None, + config=None, + update_mask=None, + retry=None, + timeout=None, + metadata=None, + ) diff --git a/tests/contrib/operators/test_gcp_dlp_operator_system.py b/tests/contrib/operators/test_gcp_dlp_operator_system.py new file mode 100644 index 0000000000000..2692de8032973 --- /dev/null +++ b/tests/contrib/operators/test_gcp_dlp_operator_system.py @@ -0,0 +1,43 @@ +# -*- coding: utf-8 -*- +# +# 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. + +# pylint: disable=C0111 +""" +This module contains various unit tests for +example_gcp_dlp DAG +""" + +import unittest + +from tests.contrib.utils.base_gcp_system_test_case import \ + SKIP_TEST_WARNING, DagGcpSystemTestCase +from tests.contrib.utils.gcp_authenticator import GCP_DLP_KEY + + +@unittest.skipIf( + DagGcpSystemTestCase.skip_check(GCP_DLP_KEY), SKIP_TEST_WARNING) +class GcpDLPExampleDagsSystemTest(DagGcpSystemTestCase): + def __init__(self, method_name='runTest'): + super().__init__( + method_name, + dag_id='example_gcp_dlp', + gcp_key=GCP_DLP_KEY) + + def test_run_example_dag_function(self): + self._run_dag() diff --git a/tests/contrib/utils/gcp_authenticator.py b/tests/contrib/utils/gcp_authenticator.py index 0b84d0d54b555..796e1b8c3c462 100644 --- a/tests/contrib/utils/gcp_authenticator.py +++ b/tests/contrib/utils/gcp_authenticator.py @@ -32,6 +32,7 @@ GCP_CLOUDSQL_KEY = 'gcp_cloudsql.json' GCP_COMPUTE_KEY = 'gcp_compute.json' GCP_DATAPROC_KEY = 'gcp_dataproc.json' +GCP_DLP_KEY = 'gcp_dlp.json' GCP_FUNCTION_KEY = 'gcp_function.json' GCP_GCS_KEY = 'gcp_gcs.json' GCP_GCS_TRANSFER_KEY = 'gcp_gcs_transfer.json'