From 7c1c4c6c22b36303cb9f59f6bcd0a5ad14521dbc Mon Sep 17 00:00:00 2001 From: Maksim Moiseenkov Date: Wed, 6 Nov 2024 15:38:36 +0000 Subject: [PATCH] Remove deprecated functionality from Google provider --- airflow/serialization/serialized_objects.py | 2 +- .../connections/gcp.rst | 7 +- .../operators/cloud/automl.rst | 2 +- .../operators/cloud/dataflow.rst | 8 +- .../operators/cloud/gcs.rst | 14 - .../operators/cloud/vertex_ai.rst | 13 +- .../marketing_platform/analytics.rst | 53 - .../howto/define-extra-link.rst | 32 +- .../airflow/providers/google/CHANGELOG.rst | 146 ++ .../airflow/providers/google/ads/hooks/ads.py | 3 +- .../providers/google/cloud/hooks/automl.py | 6 +- .../providers/google/cloud/hooks/bigquery.py | 1522 +---------------- .../google/cloud/hooks/bigquery_dts.py | 12 +- .../providers/google/cloud/hooks/bigtable.py | 6 +- .../google/cloud/hooks/cloud_batch.py | 6 +- .../google/cloud/hooks/cloud_build.py | 15 +- .../google/cloud/hooks/cloud_composer.py | 16 - .../google/cloud/hooks/cloud_memorystore.py | 8 +- .../providers/google/cloud/hooks/cloud_run.py | 12 +- .../providers/google/cloud/hooks/cloud_sql.py | 9 +- .../hooks/cloud_storage_transfer_service.py | 7 +- .../providers/google/cloud/hooks/compute.py | 6 +- .../google/cloud/hooks/compute_ssh.py | 5 - .../google/cloud/hooks/datacatalog.py | 6 +- .../providers/google/cloud/hooks/dataflow.py | 14 +- .../providers/google/cloud/hooks/dataform.py | 8 - .../google/cloud/hooks/datafusion.py | 14 +- .../providers/google/cloud/hooks/dataplex.py | 8 +- .../providers/google/cloud/hooks/dataprep.py | 4 +- .../providers/google/cloud/hooks/dataproc.py | 20 +- .../google/cloud/hooks/dataproc_metastore.py | 8 - .../providers/google/cloud/hooks/datastore.py | 6 +- .../providers/google/cloud/hooks/dlp.py | 6 +- .../providers/google/cloud/hooks/functions.py | 6 +- .../providers/google/cloud/hooks/gcs.py | 17 - .../providers/google/cloud/hooks/gdm.py | 18 +- .../providers/google/cloud/hooks/kms.py | 6 +- .../google/cloud/hooks/kubernetes_engine.py | 101 +- .../google/cloud/hooks/life_sciences.py | 6 +- .../providers/google/cloud/hooks/looker.py | 3 +- .../providers/google/cloud/hooks/mlengine.py | 8 - .../google/cloud/hooks/natural_language.py | 6 +- .../providers/google/cloud/hooks/os_login.py | 6 +- .../providers/google/cloud/hooks/pubsub.py | 6 +- .../google/cloud/hooks/secret_manager.py | 73 +- .../providers/google/cloud/hooks/spanner.py | 6 +- .../google/cloud/hooks/speech_to_text.py | 6 +- .../google/cloud/hooks/stackdriver.py | 6 +- .../providers/google/cloud/hooks/tasks.py | 6 +- .../google/cloud/hooks/text_to_speech.py | 6 +- .../providers/google/cloud/hooks/translate.py | 6 +- .../google/cloud/hooks/vertex_ai/auto_ml.py | 6 +- .../hooks/vertex_ai/batch_prediction_job.py | 6 +- .../cloud/hooks/vertex_ai/custom_job.py | 6 +- .../google/cloud/hooks/vertex_ai/dataset.py | 8 - .../cloud/hooks/vertex_ai/endpoint_service.py | 8 - .../cloud/hooks/vertex_ai/generative_model.py | 15 +- .../vertex_ai/hyperparameter_tuning_job.py | 6 +- .../cloud/hooks/vertex_ai/model_service.py | 8 - .../cloud/hooks/vertex_ai/pipeline_job.py | 1 + .../google/cloud/hooks/video_intelligence.py | 6 +- .../providers/google/cloud/hooks/vision.py | 6 +- .../providers/google/cloud/hooks/workflows.py | 8 - .../providers/google/cloud/links/dataproc.py | 1 - .../google/cloud/operators/automl.py | 4 +- .../google/cloud/operators/bigquery.py | 342 +--- .../google/cloud/operators/dataflow.py | 516 ------ .../google/cloud/operators/dataproc.py | 630 ------- .../cloud/operators/kubernetes_engine.py | 10 - .../cloud/operators/vertex_ai/auto_ml.py | 92 - .../vertex_ai/batch_prediction_job.py | 13 +- .../cloud/operators/vertex_ai/custom_job.py | 16 - .../vertex_ai/hyperparameter_tuning_job.py | 13 +- .../google/cloud/secrets/secret_manager.py | 19 +- .../google/cloud/sensors/bigquery.py | 80 - .../google/cloud/sensors/cloud_composer.py | 93 +- .../providers/google/cloud/sensors/gcs.py | 35 +- .../providers/google/cloud/triggers/gcs.py | 2 +- .../google/common/hooks/base_google.py | 12 +- .../google/common/hooks/discovery_api.py | 5 - .../marketing_platform/hooks/analytics.py | 211 --- .../hooks/campaign_manager.py | 2 - .../marketing_platform/hooks/display_video.py | 4 +- .../marketing_platform/hooks/search_ads.py | 9 +- .../marketing_platform/operators/analytics.py | 551 ------ .../operators/campaign_manager.py | 41 - .../operators/display_video.py | 46 - .../sensors/campaign_manager.py | 6 - .../sensors/display_video.py | 12 - .../airflow/providers/google/provider.yaml | 14 - .../providers/google/suite/hooks/calendar.py | 7 - .../providers/google/suite/hooks/drive.py | 5 - .../providers/google/suite/hooks/sheets.py | 6 - .../google/suite/operators/sheets.py | 6 - .../providers/google/suite/sensors/drive.py | 6 - .../google/suite/transfers/gcs_to_gdrive.py | 6 - .../google/suite/transfers/gcs_to_sheets.py | 6 - .../google/suite/transfers/local_to_drive.py | 6 - .../google/suite/transfers/sql_to_sheets.py | 6 - providers/tests/deprecations_ignore.yml | 11 - .../tests/google/cloud/hooks/test_automl.py | 5 - .../tests/google/cloud/hooks/test_bigquery.py | 759 +------- .../google/cloud/hooks/test_bigquery_dts.py | 8 - .../tests/google/cloud/hooks/test_bigtable.py | 5 - .../google/cloud/hooks/test_cloud_build.py | 8 - .../google/cloud/hooks/test_cloud_composer.py | 8 - .../cloud/hooks/test_cloud_memorystore.py | 4 - .../google/cloud/hooks/test_cloud_sql.py | 4 - .../test_cloud_storage_transfer_service.py | 4 - .../tests/google/cloud/hooks/test_compute.py | 4 - .../google/cloud/hooks/test_compute_ssh.py | 4 - .../google/cloud/hooks/test_datacatalog.py | 4 - .../tests/google/cloud/hooks/test_dataflow.py | 8 - .../tests/google/cloud/hooks/test_dataform.py | 4 - .../google/cloud/hooks/test_datafusion.py | 8 - .../tests/google/cloud/hooks/test_dataplex.py | 5 - .../tests/google/cloud/hooks/test_dataproc.py | 8 - .../cloud/hooks/test_dataproc_metastore.py | 4 - .../google/cloud/hooks/test_datastore.py | 4 - .../tests/google/cloud/hooks/test_dlp.py | 4 - .../google/cloud/hooks/test_functions.py | 4 - .../tests/google/cloud/hooks/test_gcs.py | 7 +- .../tests/google/cloud/hooks/test_gdm.py | 4 - .../tests/google/cloud/hooks/test_kms.py | 5 - .../cloud/hooks/test_kubernetes_engine.py | 14 +- .../google/cloud/hooks/test_life_sciences.py | 4 - .../tests/google/cloud/hooks/test_mlengine.py | 4 - .../cloud/hooks/test_natural_language.py | 5 - .../tests/google/cloud/hooks/test_os_login.py | 4 - .../tests/google/cloud/hooks/test_pubsub.py | 4 - .../google/cloud/hooks/test_secret_manager.py | 57 +- .../cloud/hooks/test_secret_manager_system.py | 34 +- .../tests/google/cloud/hooks/test_spanner.py | 5 - .../google/cloud/hooks/test_speech_to_text.py | 5 - .../google/cloud/hooks/test_stackdriver.py | 4 - .../tests/google/cloud/hooks/test_tasks.py | 5 - .../google/cloud/hooks/test_text_to_speech.py | 5 - .../google/cloud/hooks/test_translate.py | 6 - .../cloud/hooks/test_video_intelligence.py | 5 - .../tests/google/cloud/hooks/test_vision.py | 4 - .../google/cloud/hooks/test_workflows.py | 6 - .../cloud/hooks/vertex_ai/test_auto_ml.py | 4 - .../vertex_ai/test_batch_prediction_job.py | 4 - .../cloud/hooks/vertex_ai/test_custom_job.py | 4 - .../cloud/hooks/vertex_ai/test_dataset.py | 4 - .../hooks/vertex_ai/test_endpoint_service.py | 4 - .../test_hyperparameter_tuning_job.py | 4 - .../hooks/vertex_ai/test_model_service.py | 4 - .../google/cloud/operators/test_bigquery.py | 398 ----- .../google/cloud/operators/test_dataflow.py | 361 ---- .../google/cloud/operators/test_dataproc.py | 430 ----- .../google/cloud/operators/test_vertex_ai.py | 191 +-- .../google/cloud/sensors/test_bigquery.py | 133 +- .../cloud/sensors/test_cloud_composer.py | 64 +- .../tests/google/cloud/sensors/test_gcs.py | 51 +- .../cloud/triggers/test_kubernetes_engine.py | 1 - .../tests/google/cloud/utils/base_gcp_mock.py | 4 - .../google/common/hooks/test_base_google.py | 25 - .../hooks/test_analytics.py | 191 --- .../operators/test_analytics.py | 273 --- .../operators/test_campaign_manager.py | 5 - .../operators/test_display_video.py | 14 - .../sensors/test_campaign_manager.py | 1 - .../sensors/test_display_video.py | 2 - .../tests/google/suite/sensors/test_drive.py | 3 - .../suite/transfers/test_gcs_to_gdrive.py | 9 - .../suite/transfers/test_gcs_to_sheets.py | 3 - .../suite/transfers/test_sql_to_sheets.py | 1 - .../google/cloud/gcs/example_gcs_sensor.py | 10 - .../vertex_ai/example_vertex_ai_custom_job.py | 2 - ...ple_vertex_ai_hyperparameter_tuning_job.py | 2 - .../example_vertex_ai_model_service.py | 2 - tests/always/test_project_structure.py | 20 - 173 files changed, 341 insertions(+), 8160 deletions(-) delete mode 100644 docs/apache-airflow-providers-google/operators/marketing_platform/analytics.rst delete mode 100644 providers/src/airflow/providers/google/marketing_platform/hooks/analytics.py delete mode 100644 providers/src/airflow/providers/google/marketing_platform/operators/analytics.py delete mode 100644 providers/tests/google/marketing_platform/hooks/test_analytics.py delete mode 100644 providers/tests/google/marketing_platform/operators/test_analytics.py diff --git a/airflow/serialization/serialized_objects.py b/airflow/serialization/serialized_objects.py index 4b7ee6d0871b..12adf6a7f50f 100644 --- a/airflow/serialization/serialized_objects.py +++ b/airflow/serialization/serialized_objects.py @@ -1547,7 +1547,7 @@ def _serialize_operator_extra_links(cls, operator_extra_links: Iterable[BaseOper Store the import path of the OperatorLink and the arguments passed to it. For example: - ``[{'airflow.providers.google.cloud.operators.bigquery.BigQueryConsoleLink': {}}]`` + ``[{'airflow.providers.google.cloud.links.bigquery.BigQueryDatasetLink': {}}]`` :param operator_extra_links: Operator Link :return: Serialized Operator Link diff --git a/docs/apache-airflow-providers-google/connections/gcp.rst b/docs/apache-airflow-providers-google/connections/gcp.rst index 541853195743..62ef99e7d394 100644 --- a/docs/apache-airflow-providers-google/connections/gcp.rst +++ b/docs/apache-airflow-providers-google/connections/gcp.rst @@ -207,11 +207,6 @@ In order for this example to work, the account ``impersonated_account`` must gra access token, which will allow to act on its behalf using its permissions. ``impersonated_account`` does not even need to have a generated key. -.. warning:: - :class:`~airflow.providers.google.cloud.operators.dataflow.DataflowCreateJavaJobOperator` and - :class:`~airflow.providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator` - do not support direct impersonation as of now. - In case of operators that connect to multiple Google services, all hooks use the same value of ``impersonation_chain`` (if applicable). You can also impersonate accounts from projects other than the project of the originating account. In that case, the project id of the impersonated @@ -306,7 +301,7 @@ For example: task_id="create-spreadsheet", gcp_conn_id="google_cloud_default", spreadsheet=SPREADSHEET, - delegate_to=f"projects/-/serviceAccounts/SA@{PROJECT_ID}.iam.gserviceaccount.com", + impersonation_chain=f"projects/-/serviceAccounts/SA@{PROJECT_ID}.iam.gserviceaccount.com", ) Note that as domain-wide delegation is currently supported by most of the Google operators and hooks, its usage should be limited only to Google Workspace (gsuite) and marketing platform operators and hooks or by accessing these services through the GoogleDiscoveryAPI hook. It is deprecated in the following usages: diff --git a/docs/apache-airflow-providers-google/operators/cloud/automl.rst b/docs/apache-airflow-providers-google/operators/cloud/automl.rst index ebfe3ca501f4..654589ed1404 100644 --- a/docs/apache-airflow-providers-google/operators/cloud/automl.rst +++ b/docs/apache-airflow-providers-google/operators/cloud/automl.rst @@ -105,7 +105,7 @@ returns the id of model in :ref:`XCom ` under ``model_id`` key. This operator is deprecated when running for text, video and vision prediction and will be removed soon. All the functionality of legacy AutoML Natural Language, Vision, Video Intelligence and new features are available on the Vertex AI platform. Please use -:class:`~airflow.providers.google.cloud.operators.vertex_ai.auto_ml.CreateAutoMLTextTrainingJobOperator`, +:class:`~airflow.providers.google.cloud.operators.vertex_ai.generative_model.SupervisedFineTuningTrainOperator`, :class:`~airflow.providers.google.cloud.operators.vertex_ai.auto_ml.CreateAutoMLImageTrainingJobOperator` or :class:`~airflow.providers.google.cloud.operators.vertex_ai.auto_ml.CreateAutoMLVideoTrainingJobOperator`. diff --git a/docs/apache-airflow-providers-google/operators/cloud/dataflow.rst b/docs/apache-airflow-providers-google/operators/cloud/dataflow.rst index 21895e4e6e67..3213aec60690 100644 --- a/docs/apache-airflow-providers-google/operators/cloud/dataflow.rst +++ b/docs/apache-airflow-providers-google/operators/cloud/dataflow.rst @@ -37,8 +37,8 @@ There are several ways to run a Dataflow pipeline depending on your environment, dependencies must be installed on the worker. For Java, worker must have the JRE Runtime installed. For Python, the Python interpreter. The runtime versions must be compatible with the pipeline versions. This is the fastest way to start a pipeline, but because of its frequent problems with system dependencies, - it may cause problems. See: :ref:`howto/operator:DataflowCreateJavaJobOperator`, - :ref:`howto/operator:DataflowCreatePythonJobOperator` for more detailed information. + it may cause problems. See: :ref:`howto/operator:JavaSDKPipelines`, + :ref:`howto/operator:PythonSDKPipelines` for more detailed information. Developer can also create a pipeline by passing its structure in a JSON format and then run it to create a Job. See: :ref:`howto/operator:DataflowCreatePipelineOperator` and :ref:`howto/operator:DataflowRunPipelineOperator` @@ -107,7 +107,7 @@ the create job operators. The source file can be located on GCS or on the local or :class:`~airflow.providers.apache.beam.operators.beam.BeamRunPythonPipelineOperator` -.. _howto/operator:DataflowCreateJavaJobOperator: +.. _howto/operator:JavaSDKPipelines: Java SDK pipelines """""""""""""""""" @@ -141,7 +141,7 @@ Here is an example of creating and running a pipeline in Java with jar stored on :start-after: [START howto_operator_start_java_job_local_jar] :end-before: [END howto_operator_start_java_job_local_jar] -.. _howto/operator:DataflowCreatePythonJobOperator: +.. _howto/operator:PythonSDKPipelines: Python SDK pipelines """""""""""""""""""" diff --git a/docs/apache-airflow-providers-google/operators/cloud/gcs.rst b/docs/apache-airflow-providers-google/operators/cloud/gcs.rst index 82d38fda87b8..3d14d0a37e8e 100644 --- a/docs/apache-airflow-providers-google/operators/cloud/gcs.rst +++ b/docs/apache-airflow-providers-google/operators/cloud/gcs.rst @@ -174,20 +174,6 @@ Also you can use deferrable mode in this operator if you would like to free up t :start-after: [START howto_sensor_object_exists_task_defered] :end-before: [END howto_sensor_object_exists_task_defered] -.. _howto/sensor:GCSObjectExistenceAsyncSensor: - -GCSObjectExistenceAsyncSensor ------------------------------ - -:class:`~airflow.providers.google.cloud.sensors.gcs.GCSObjectExistenceAsyncSensor` is deprecated and will be removed in a future release. Please use :class:`~airflow.providers.google.cloud.sensors.gcs.GCSObjectExistenceSensor` and use the deferrable mode in that operator. - -.. exampleinclude:: /../../providers/tests/system/google/cloud/gcs/example_gcs_sensor.py - :language: python - :dedent: 4 - :start-after: [START howto_sensor_object_exists_task_async] - :end-before: [END howto_sensor_object_exists_task_async] - - .. _howto/sensor:GCSObjectsWithPrefixExistenceSensor: GCSObjectsWithPrefixExistenceSensor diff --git a/docs/apache-airflow-providers-google/operators/cloud/vertex_ai.rst b/docs/apache-airflow-providers-google/operators/cloud/vertex_ai.rst index f24ab3b06f71..f8f87040f9fc 100644 --- a/docs/apache-airflow-providers-google/operators/cloud/vertex_ai.rst +++ b/docs/apache-airflow-providers-google/operators/cloud/vertex_ai.rst @@ -216,7 +216,7 @@ To create a Google Vertex AI Auto ML training jobs you have five operators :class:`~airflow.providers.google.cloud.operators.vertex_ai.auto_ml.CreateAutoMLForecastingTrainingJobOperator` :class:`~airflow.providers.google.cloud.operators.vertex_ai.auto_ml.CreateAutoMLImageTrainingJobOperator` :class:`~airflow.providers.google.cloud.operators.vertex_ai.auto_ml.CreateAutoMLTabularTrainingJobOperator` -:class:`~airflow.providers.google.cloud.operators.vertex_ai.auto_ml.CreateAutoMLTextTrainingJobOperator` +:class:`~airflow.providers.google.cloud.operators.vertex_ai.generative_model.SupervisedFineTuningTrainOperator` :class:`~airflow.providers.google.cloud.operators.vertex_ai.auto_ml.CreateAutoMLVideoTrainingJobOperator` Each of them will wait for the operation to complete. The results of each operator will be a model which was trained by user using these operators. @@ -257,17 +257,6 @@ put dataset id to ``dataset_id`` parameter in operator. :start-after: [START how_to_cloud_vertex_ai_create_auto_ml_tabular_training_job_operator] :end-before: [END how_to_cloud_vertex_ai_create_auto_ml_tabular_training_job_operator] -How to run AutoML Text Training Job -:class:`~airflow.providers.google.cloud.operators.vertex_ai.auto_ml.CreateAutoMLTextTrainingJobOperator` - -Operator is deprecated, the non-training (existing legacy models) AutoMLText API will be deprecated -on June 15, 2025. -There are 2 options for text classification, extraction, and sentiment analysis tasks replacement: -- Prompts with pre-trained Gemini model, using :class:`~airflow.providers.google.cloud.operators.vertex_ai.generative_model.TextGenerationModelPredictOperator`. -- Fine tuning over Gemini model, For more tailored results, using :class:`~airflow.providers.google.cloud.operators.vertex_ai.generative_model.SupervisedFineTuningTrainOperator`. - -Please visit the https://cloud.google.com/vertex-ai/generative-ai/docs/models/gemini-tuning for more details. - How to run AutoML Video Training Job :class:`~airflow.providers.google.cloud.operators.vertex_ai.auto_ml.CreateAutoMLVideoTrainingJobOperator` diff --git a/docs/apache-airflow-providers-google/operators/marketing_platform/analytics.rst b/docs/apache-airflow-providers-google/operators/marketing_platform/analytics.rst deleted file mode 100644 index 0888e81a3473..000000000000 --- a/docs/apache-airflow-providers-google/operators/marketing_platform/analytics.rst +++ /dev/null @@ -1,53 +0,0 @@ - .. Licensed to the Apache Software Foundation (ASF) under one - or more contributor license agreements. See the NOTICE file - distributed with this work for additional information - regarding copyright ownership. The ASF licenses this file - to you under the Apache License, Version 2.0 (the - "License"); you may not use this file except in compliance - with the License. You may obtain a copy of the License at - - .. http://www.apache.org/licenses/LICENSE-2.0 - - .. Unless required by applicable law or agreed to in writing, - software distributed under the License is distributed on an - "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - KIND, either express or implied. See the License for the - specific language governing permissions and limitations - under the License. - -Google Analytics 360 Operators -============================== - -Google Analytics 360 operators allow you to lists all accounts to which the user has access. -For more information about the Google Analytics 360 API check -`official documentation `__. - -Please note that the Google Analytics 360 API is replaced by -`Google Analytics 4 `__ and is -`turned down on July 1, 2024 `__. -Thus consider using new :doc:`Google Analytics (GA4) Admin Operators `. - -Prerequisite Tasks -^^^^^^^^^^^^^^^^^^ - -.. include:: /operators/_partials/prerequisite_tasks.rst - -List the Accounts -^^^^^^^^^^^^^^^^^ - -To list accounts from Analytics you can use the -:class:`~airflow.providers.google.marketing_platform.operators.analytics.GoogleAnalyticsListAccountsOperator`. - -Get Ad Words Link -^^^^^^^^^^^^^^^^^ - -Returns a web property-Google Ads link to which the user has access. -To list web property-Google Ads link you can use the -:class:`~airflow.providers.google.marketing_platform.operators.analytics.GoogleAnalyticsGetAdsLinkOperator`. - -List Google Ads Links -^^^^^^^^^^^^^^^^^^^^^ - -Operator returns a list of entity Google Ads links. -To list Google Ads links you can use the -:class:`~airflow.providers.google.marketing_platform.operators.analytics.GoogleAnalyticsRetrieveAdsLinksListOperator`. diff --git a/docs/apache-airflow/howto/define-extra-link.rst b/docs/apache-airflow/howto/define-extra-link.rst index 51df8535698a..14864d145003 100644 --- a/docs/apache-airflow/howto/define-extra-link.rst +++ b/docs/apache-airflow/howto/define-extra-link.rst @@ -138,24 +138,34 @@ Console, but if we wanted to change that link we could: BIGQUERY_JOB_DETAILS_LINK_FMT = "http://console.cloud.google.com/bigquery?j={job_id}" - class BigQueryConsoleLink(BaseOperatorLink): + class BigQueryDatasetLink(BaseGoogleLink): """ - Helper class for constructing BigQuery link. + Helper class for constructing BigQuery Dataset Link. """ - name = "BigQuery Console" - operators = [BigQueryOperator] + name = "BigQuery Dataset" + key = "bigquery_dataset" + format_str = BIGQUERY_DATASET_LINK - def get_link(self, operator: BaseOperator, *, ti_key: TaskInstanceKey): - job_id = XCom.get_one(ti_key=ti_key, key="job_id") - return BIGQUERY_JOB_DETAILS_LINK_FMT.format(job_id=job_id) if job_id else "" + @staticmethod + def persist( + context: Context, + task_instance: BaseOperator, + dataset_id: str, + project_id: str, + ): + task_instance.xcom_push( + context, + key=BigQueryDatasetLink.key, + value={"dataset_id": dataset_id, "project_id": project_id}, + ) # Defining the plugin class class AirflowExtraLinkPlugin(AirflowPlugin): name = "extra_link_plugin" operator_extra_links = [ - BigQueryConsoleLink(), + BigQueryDatasetLink(), ] @@ -171,9 +181,7 @@ by ``apache-airflow-providers-google`` provider currently: .. code-block:: yaml extra-links: - - airflow.providers.google.cloud.operators.bigquery.BigQueryConsoleLink - - airflow.providers.google.cloud.operators.bigquery.BigQueryConsoleIndexableLink - - airflow.providers.google.cloud.operators.mlengine.AIPlatformConsoleLink - + - airflow.providers.google.cloud.links.bigquery.BigQueryDatasetLink + - airflow.providers.google.cloud.links.bigquery.BigQueryTableLink You can include as many operators with extra links as you want. diff --git a/providers/src/airflow/providers/google/CHANGELOG.rst b/providers/src/airflow/providers/google/CHANGELOG.rst index bc407c365509..1d0227056161 100644 --- a/providers/src/airflow/providers/google/CHANGELOG.rst +++ b/providers/src/airflow/providers/google/CHANGELOG.rst @@ -27,6 +27,152 @@ Changelog --------- +11.0.0 +...... + +Breaking changes +~~~~~~~~~~~~~~~~ + +.. warning:: + Deprecated classes, parameters and features have been removed from the Google provider package. + The following breaking changes were introduced: + + * Operators + + * Removed ``CreateAutoMLTextTrainingJobOperator``. Use ``SupervisedFineTuningTrainOperator`` instead + * Removed ``BigQueryExecuteQueryOperator``. Use ``BigQueryInsertJobOperator`` instead + * Removed ``BigQueryPatchDatasetOperator``. Use ``BigQueryUpdateDatasetOperator`` instead + * Removed ``DataflowCreateJavaJobOperator``. Use ``BeamRunJavaPipelineOperator`` instead + * Removed ``DataflowCreatePythonJobOperator``. Use ``BeamRunPythonPipelineOperator`` instead + * Removed ``DataprocSubmitPigJobOperator``. Use ``DataprocSubmitJobOperator`` instead + * Removed ``DataprocSubmitHiveJobOperator``. Use ``DataprocSubmitJobOperator`` instead + * Removed ``DataprocSubmitSparkSqlJobOperator``. Use ``DataprocSubmitJobOperator`` instead + * Removed ``DataprocSubmitSparkJobOperator``. Use ``DataprocSubmitJobOperator`` instead + * Removed ``DataprocSubmitHadoopJobOperator``. Use ``DataprocSubmitJobOperator`` instead + * Removed ``DataprocSubmitPySparkJobOperator``. Use ``DataprocSubmitJobOperator`` instead + * Removed ``GoogleAnalyticsListAccountsOperator``. Use ``GoogleAnalyticsAdminListAccountsOperator`` instead + * Removed ``GoogleAnalyticsGetAdsLinkOperator``. Use ``GoogleAnalyticsAdminGetGoogleAdsLinkOperator`` instead + * Removed ``GoogleAnalyticsRetrieveAdsLinksListOperator``. Use ``GoogleAnalyticsAdminListGoogleAdsLinksOperator`` instead + * Removed ``GoogleAnalyticsDataImportUploadOperator``. Use ``GoogleAnalyticsAdminCreateDataStreamOperator`` instead + * Removed ``GoogleAnalyticsDeletePreviousDataUploadsOperator``. Use ``GoogleAnalyticsAdminDeleteDataStreamOperator`` instead + * Removed ``GoogleAnalyticsModifyFileHeadersDataImportOperator``. The class is no longer in actual use due to + Google Analytics API v3 has reached sunset, and thus the covered use case is no longer relevant + * Removed ``GoogleCampaignManagerDeleteReportOperator.delegate_to``. Use ``GoogleCampaignManagerDeleteReportOperator.impersonation_chain`` instead + * Removed ``GoogleCampaignManagerDownloadReportOperator.delegate_to``. Use ``GoogleCampaignManagerDownloadReportOperator.impersonation_chain`` instead + * Removed ``GoogleCampaignManagerInsertReportOperator.delegate_to``. Use ``GoogleCampaignManagerInsertReportOperator.impersonation_chain`` instead + * Removed ``GoogleCampaignManagerRunReportOperator.delegate_to``. Use ``GoogleCampaignManagerRunReportOperator.impersonation_chain`` instead + * Removed ``GoogleCampaignManagerBatchInsertConversionsOperator.delegate_to``. Use ``GoogleCampaignManagerBatchInsertConversionsOperator.impersonation_chain`` instead + * Removed ``GoogleCampaignManagerBatchUpdateConversionsOperator.delegate_to``. Use ``GoogleCampaignManagerBatchUpdateConversionsOperator.impersonation_chain`` instead + * Removed ``GoogleDisplayVideo360CreateQueryOperator.delegate_to``. Use ``GoogleDisplayVideo360CreateQueryOperator.impersonation_chain`` instead + * Removed ``GoogleDisplayVideo360DeleteReportOperator.delegate_to``. Use ``GoogleDisplayVideo360DeleteReportOperator.impersonation_chain`` instead + * Removed ``GoogleDisplayVideo360DownloadReportV2Operator.delegate_to``. Use ``GoogleDisplayVideo360DownloadReportV2Operator.impersonation_chain`` instead + * Removed ``GoogleDisplayVideo360RunQueryOperator.delegate_to``. Use ``GoogleDisplayVideo360RunQueryOperator.impersonation_chain`` instead + * Removed ``GoogleDisplayVideo360DownloadLineItemsOperator.delegate_to``. Use ``GoogleDisplayVideo360DownloadLineItemsOperator.impersonation_chain`` instead + * Removed ``GoogleDisplayVideo360UploadLineItemsOperator.delegate_to``. Use ``GoogleDisplayVideo360UploadLineItemsOperator.impersonation_chain`` instead + * Removed ``GoogleDisplayVideo360CreateSDFDownloadTaskOperator.delegate_to``. Use ``GoogleDisplayVideo360CreateSDFDownloadTaskOperator.impersonation_chain`` instead + * Removed ``GoogleDisplayVideo360SDFtoGCSOperator.delegate_to``. Use ``GoogleDisplayVideo360SDFtoGCSOperator.impersonation_chain`` instead + * Removed ``GoogleSheetsCreateSpreadsheetOperator.delegate_to``. Use ``GoogleSheetsCreateSpreadsheetOperator.impersonation_chain`` instead + * Removed ``GCSToGoogleDriveOperator.delegate_to``. Use ``GCSToGoogleDriveOperator.impersonation_chain`` instead + * Removed ``GCSToGoogleSheetsOperator.delegate_to``. Use ``GCSToGoogleSheetsOperator.impersonation_chain`` instead + * Removed ``LocalFilesystemToGoogleDriveOperator.delegate_to``. Use ``LocalFilesystemToGoogleDriveOperator.impersonation_chain`` instead + * Removed ``SQLToGoogleSheetsOperator.delegate_to``. Use ``SQLToGoogleSheetsOperator.impersonation_chain`` instead + * Removed ``CreateBatchPredictionJobOperator.sync``. This parameter is not in actual use + * Removed ``CreateHyperparameterTuningJobOperator.sync``. This parameter is not in actual use + * Removed ``CustomTrainingJobBaseOperator.sync``. This parameter is not in actual use + * Removed ``GKEStartPodOperator.get_gke_config_file()``. Please use ``GKEStartPodOperator.fetch_cluster_info()`` instead + + * Triggers + + * Removed support of ``delegate_to`` field in the ``GCSCheckBlobUpdateTimeTrigger.hook_params`` parameter + + * Sensors + + * Removed ``BigQueryTableExistenceAsyncSensor``. Use ``BigQueryTableExistenceSensor`` and set deferrable attribute + to True instead + * Removed ``BigQueryTableExistencePartitionAsyncSensor``. Use ``BigQueryTablePartitionExistenceSensor`` and set + deferrable attribute to True instead + * Removed ``CloudComposerEnvironmentSensor``. Use ``CloudComposerCreateEnvironmentOperator``, + ``CloudComposerUpdateEnvironmentOperator``, or ``CloudComposerDeleteEnvironmentOperator`` instead + * Removed ``GCSObjectExistenceAsyncSensor``. Use ``GCSObjectExistenceSensor`` and set deferrable attribute + to True instead + * Removed ``GoogleCampaignManagerReportSensor.delegate_to``. Use ``GoogleCampaignManagerReportSensor.impersonation_chain`` instead + * Removed ``GoogleDisplayVideo360GetSDFDownloadOperationSensor.delegate_to``. Use ``GoogleDisplayVideo360GetSDFDownloadOperationSensor.impersonation_chain`` instead + * Removed ``GoogleDisplayVideo360RunQuerySensor.delegate_to``. Use ``GoogleDisplayVideo360RunQuerySensor.impersonation_chain`` instead + * Removed ``GoogleDriveFileExistenceSensor.delegate_to``. Use ``GoogleDriveFileExistenceSensor.impersonation_chain`` instead + + * Links + + * Removed ``BigQueryConsoleIndexableLink``. This property is no longer in actual use + * Removed ``BigQueryConsoleLink``. This property is no longer in actual use + + * Hooks + + * Removed ``GKEDeploymentHook``. Use ``GKEKubernetesHook`` instead + * Removed ``GKECustomResourceHook``. Use ``GKEKubernetesHook`` instead + * Removed ``GKEPodHook``. Use ``GKEKubernetesHook`` instead + * Removed ``GKEJobHook``. Use ``GKEKubernetesHook`` instead + * Removed ``GKEPodAsyncHook``. Use ``GKEKubernetesAsyncHook`` instead + * Removed ``SecretsManagerHook``. Use ``GoogleCloudSecretManagerHook`` instead + * Removed ``GoogleAnalyticsHook``. The class is no longer in actual use due to Google Analytics API v3 has reached + sunset + * Removed ``GoogleBaseHook.delegate_to``. Please use ``GoogleBaseHook.impersonation_chain``. Please note that + the ``delegate_to`` parameter used to be inherited in all Google hooks, and from now its support is removed + everywhere within the Google provider + * Removed ``GoogleDiscoveryApiHook.delegate_to``. Please use ``GoogleDiscoveryApiHook.impersonation_chain`` instead + * Removed ``GoogleCampaignManagerHook.delegate_to``. Please use ``GoogleCampaignManagerHook.impersonation_chain`` instead + * Removed ``GoogleDisplayVideo360Hook.delegate_to``. Please use ``GoogleDisplayVideo360Hook.impersonation_chain`` instead + * Removed ``GoogleSearchAdsHook.delegate_to``. Please use ``GoogleSearchAdsHook.impersonation_chain`` instead + * Removed ``GoogleCalendarHook.delegate_to``. Please use ``GoogleCalendarHook.impersonation_chain`` instead + * Removed ``GoogleDriveHook.delegate_to``. Please use ``GoogleDriveHook.impersonation_chain`` instead + * Removed ``GSheetsHook.delegate_to``. Please use ``GSheetsHook.impersonation_chain`` instead + * Removed ``BigQueryHook.credentials_path``. This property is no longer in actual use + * Removed ``GKEHook.get_conn()``. Please use ``GKEHook.get_cluster_manager_client()`` instead + * Removed ``GKEHook.get_client()``. Please use ``GKEHook.get_cluster_manager_client()`` instead + * Removed ``BigQueryHook.patch_table()``. Please use ``BigQueryHook.update_table()`` instead + * Removed ``BigQueryHook.patch_dataset()``. Please use ``BigQueryHook.update_dataset()`` instead + * Removed ``BigQueryHook.get_dataset_tables_list()``. Please use ``BigQueryHook.get_dataset_tables()`` instead + * Removed ``BigQueryHook.run_table_delete()``. Please use ``BigQueryHook.delete_table()`` instead + * Removed ``BigQueryHook.get_tabledata()``. Please use ``BigQueryHook.list_rows()`` instead + * Removed ``BigQueryHook.cancel_query()``. Please use ``BigQueryHook.cancel_job()`` instead + * Removed ``BigQueryHook.run_with_configuration()``. Please use ``BigQueryHook.insert_job()`` instead + * Removed ``BigQueryHook.run_load()``. Please use ``BigQueryHook.insert_job()`` instead + * Removed ``BigQueryHook.run_copy()``. Please use ``BigQueryHook.insert_job()`` instead + * Removed ``BigQueryHook.run_extract()``. Please use ``BigQueryHook.insert_job()`` instead + * Removed ``BigQueryHook.run_query()``. Please use ``BigQueryHook.insert_job()`` instead + * Removed ``BigQueryHook.create_external_table()``. Please use ``BigQueryHook.create_empty_table()`` instead + * Removed ``BigQueryHook.get_service()``. Please use ``BigQueryHook.get_client()`` instead + + * Backends + + * Removed ``CloudSecretManagerBackend.get_conn_uri()``. Please use ``CloudSecretManagerBackend.get_conn_value()`` instead + + * Other deprecations + + * Removed ``BigQueryBaseCursor.create_empty_table()``. Please use ``BigQueryHook.create_empty_table()`` instead + * Removed ``BigQueryBaseCursor.create_empty_dataset()``. Please use ``BigQueryHook.create_empty_dataset()`` instead + * Removed ``BigQueryBaseCursor.get_dataset_tables()``. Please use ``BigQueryHook.get_dataset_tables()`` instead + * Removed ``BigQueryBaseCursor.delete_dataset()``. Please use ``BigQueryHook.delete_dataset()`` instead + * Removed ``BigQueryBaseCursor.create_external_table()``. Please use ``BigQueryHook.create_empty_table()`` instead + * Removed ``BigQueryBaseCursor.patch_table()``. Please use ``BigQueryHook.update_table()`` instead + * Removed ``BigQueryBaseCursor.insert_all()``. Please use ``BigQueryHook.insert_all()`` instead + * Removed ``BigQueryBaseCursor.update_dataset()``. Please use ``BigQueryHook.update_dataset()`` instead + * Removed ``BigQueryBaseCursor.patch_dataset()``. Please use ``BigQueryHook.update_dataset()`` instead + * Removed ``BigQueryBaseCursor.get_dataset_tables_list()``. Please use ``BigQueryHook.get_dataset_tables()`` instead + * Removed ``BigQueryBaseCursor.get_datasets_list()``. Please use ``BigQueryHook.get_datasets_list()`` instead + * Removed ``BigQueryBaseCursor.get_dataset()``. Please use ``BigQueryHook.get_dataset()`` instead + * Removed ``BigQueryBaseCursor.run_grant_dataset_view_access()``. Please use ``BigQueryHook.run_grant_dataset_view_access()`` instead + * Removed ``BigQueryBaseCursor.run_table_upsert()``. Please use ``BigQueryHook.run_table_upsert()`` instead + * Removed ``BigQueryBaseCursor.run_table_delete()``. Please use ``BigQueryHook.delete_table()`` instead + * Removed ``BigQueryBaseCursor.get_tabledata()``. Please use ``BigQueryHook.list_rows()`` instead + * Removed ``BigQueryBaseCursor.get_schema()``. Please use ``BigQueryHook.get_schema()`` instead + * Removed ``BigQueryBaseCursor.poll_job_complete()``. Please use ``BigQueryHook.poll_job_complete()`` instead + * Removed ``BigQueryBaseCursor.cancel_query()``. Please use ``BigQueryHook.cancel_job()`` instead + * Removed ``BigQueryBaseCursor.run_with_configuration()``. Please use ``BigQueryHook.insert_job()`` instead + * Removed ``BigQueryBaseCursor.run_load()``. Please use ``BigQueryHook.insert_job()`` instead + * Removed ``BigQueryBaseCursor.run_copy()``. Please use ``BigQueryHook.insert_job()`` instead + * Removed ``BigQueryBaseCursor.run_extract()``. Please use ``BigQueryHook.insert_job()`` instead + * Removed ``BigQueryBaseCursor.run_query()``. Please use ``BigQueryHook.insert_job()`` instead + 10.26.0 ....... diff --git a/providers/src/airflow/providers/google/ads/hooks/ads.py b/providers/src/airflow/providers/google/ads/hooks/ads.py index 20a40e34ca5d..f8703837cc4c 100644 --- a/providers/src/airflow/providers/google/ads/hooks/ads.py +++ b/providers/src/airflow/providers/google/ads/hooks/ads.py @@ -106,8 +106,9 @@ def __init__( api_version: str | None = None, gcp_conn_id: str = "google_cloud_default", google_ads_conn_id: str = "google_ads_default", + **kwargs, ) -> None: - super().__init__() + super().__init__(**kwargs) self.api_version = api_version self.gcp_conn_id = gcp_conn_id self.google_ads_conn_id = google_ads_conn_id diff --git a/providers/src/airflow/providers/google/cloud/hooks/automl.py b/providers/src/airflow/providers/google/cloud/hooks/automl.py index 3edd4ab1dbfd..52c69dbfa229 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/automl.py +++ b/providers/src/airflow/providers/google/cloud/hooks/automl.py @@ -71,14 +71,10 @@ def __init__( impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) super().__init__( gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, + **kwargs, ) self._client: AutoMlClient | None = None diff --git a/providers/src/airflow/providers/google/cloud/hooks/bigquery.py b/providers/src/airflow/providers/google/cloud/hooks/bigquery.py index b1a7dd2e703f..159a8f3f639b 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/bigquery.py +++ b/providers/src/airflow/providers/google/cloud/hooks/bigquery.py @@ -28,7 +28,6 @@ import uuid from copy import deepcopy from datetime import datetime, timedelta -from functools import cached_property from typing import TYPE_CHECKING, Any, Iterable, Mapping, NoReturn, Sequence, Union, cast from aiohttp import ClientSession as ClientSession @@ -37,7 +36,6 @@ DEFAULT_RETRY, Client, CopyJob, - ExternalConfig, ExtractJob, LoadJob, QueryJob, @@ -47,14 +45,13 @@ from google.cloud.bigquery.dataset import AccessEntry, Dataset, DatasetListItem, DatasetReference from google.cloud.bigquery.retry import DEFAULT_JOB_RETRY from google.cloud.bigquery.table import ( - EncryptionConfiguration, Row, RowIterator, Table, TableReference, ) from google.cloud.exceptions import NotFound -from googleapiclient.discovery import Resource, build +from googleapiclient.discovery import build from pandas_gbq import read_gbq from pandas_gbq.gbq import GbqConnector # noqa: F401 used in ``airflow.contrib.hooks.bigquery`` from requests import Session @@ -159,11 +156,6 @@ def __init__( labels: dict | None = None, **kwargs, ) -> None: - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) super().__init__(**kwargs) self.use_legacy_sql: bool = self._get_field("use_legacy_sql", use_legacy_sql) self.location: str | None = self._get_field("location", location) @@ -173,15 +165,6 @@ def __init__( self.labels = self._get_field("labels", labels or {}) self.impersonation_scopes: str | Sequence[str] | None = impersonation_scopes - @cached_property - @deprecated( - planned_removal_date="November 01, 2024", - reason="This property is no longer in actual use. ", - category=AirflowProviderDeprecationWarning, - ) - def credentials_path(self) -> str: - return "bigquery_hook_credentials.json" - def get_conn(self) -> BigQueryConnection: """Get a BigQuery PEP 249 connection object.""" http_authorized = self._authorize() @@ -195,16 +178,6 @@ def get_conn(self) -> BigQueryConnection: hook=self, ) - @deprecated( - planned_removal_date="November 01, 2024", - use_instead="airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_client.", - category=AirflowProviderDeprecationWarning, - ) - def get_service(self) -> Resource: - """Get a BigQuery service object. Deprecated.""" - http_authorized = self._authorize() - return build("bigquery", "v2", http=http_authorized, cache_discovery=False) - def get_client(self, project_id: str = PROVIDE_PROJECT_ID, location: str | None = None) -> Client: """ Get an authenticated BigQuery Client. @@ -602,165 +575,6 @@ def delete_dataset( not_found_ok=True, ) - @deprecated( - planned_removal_date="November 01, 2024", - use_instead="airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.create_empty_table", - instructions="Use the replacement method with passing the `table_resource` object. " - "This gives more flexibility.", - category=AirflowProviderDeprecationWarning, - ) - @GoogleBaseHook.fallback_to_default_project_id - def create_external_table( - self, - external_project_dataset_table: str, - schema_fields: list, - source_uris: list, - source_format: str = "CSV", - autodetect: bool = False, - compression: str = "NONE", - ignore_unknown_values: bool = False, - max_bad_records: int = 0, - skip_leading_rows: int = 0, - field_delimiter: str = ",", - quote_character: str | None = None, - allow_quoted_newlines: bool = False, - allow_jagged_rows: bool = False, - encoding: str = "UTF-8", - src_fmt_configs: dict | None = None, - labels: dict | None = None, - description: str | None = None, - encryption_configuration: dict | None = None, - location: str | None = None, - project_id: str = PROVIDE_PROJECT_ID, - ) -> Table: - """ - Create an external table in the dataset with data from Google Cloud Storage. - - .. seealso:: https://cloud.google.com/bigquery/docs/reference/rest/v2/tables#resource - - This method is deprecated. Please use :func:`.create_empty_table` with - the ``table_resource`` object. See function documentation for more - details about these parameters. - - :param external_project_dataset_table: - The dotted ``(.|:).($)`` BigQuery - table name to create external table. - If ```` is not included, project will be the - project defined in the connection json. - :param schema_fields: The schema field list as defined here: - https://cloud.google.com/bigquery/docs/reference/rest/v2/tables#resource - :param source_uris: The source Google Cloud - Storage URI (e.g. gs://some-bucket/some-file.txt). A single wild - per-object name can be used. - :param source_format: File format to export. - :param autodetect: Try to detect schema and format options automatically. - Any option specified explicitly will be honored. - :param compression: [Optional] The compression type of the data source. - Possible values include GZIP and NONE. - The default value is NONE. - This setting is ignored for Google Cloud Bigtable, - Google Cloud Datastore backups and Avro formats. - :param ignore_unknown_values: [Optional] Indicates if BigQuery should allow - extra values that are not represented in the table schema. - If true, the extra values are ignored. If false, records with extra columns - are treated as bad records, and if there are too many bad records, an - invalid error is returned in the job result. - :param max_bad_records: The maximum number of bad records that BigQuery can - ignore when running the job. - :param skip_leading_rows: Number of rows to skip when loading from a CSV. - :param field_delimiter: The delimiter to use when loading from a CSV. - :param quote_character: The value that is used to quote data sections in a CSV - file. - :param allow_quoted_newlines: Whether to allow quoted newlines (true) or not - (false). - :param allow_jagged_rows: Accept rows that are missing trailing optional columns. - The missing values are treated as nulls. If false, records with missing - trailing columns are treated as bad records, and if there are too many bad - records, an invalid error is returned in the job result. Only applicable when - source_format is CSV. - :param encoding: The character encoding of the data. See: - - .. seealso:: - https://cloud.google.com/bigquery/docs/reference/rest/v2/tables#externalDataConfiguration.csvOptions.encoding - :param src_fmt_configs: configure optional fields specific to the source format - :param labels: A dictionary containing labels for the BiqQuery table. - :param description: A string containing the description for the BigQuery table. - :param encryption_configuration: [Optional] Custom encryption configuration (e.g., Cloud KMS keys). - - .. code-block:: python - - encryption_configuration = { - "kmsKeyName": "projects/testp/locations/us/keyRings/test-kr/cryptoKeys/test-key", - } - """ - location = location or self.location - src_fmt_configs = src_fmt_configs or {} - source_format = source_format.upper() - compression = compression.upper() - - external_config_api_repr = { - "autodetect": autodetect, - "sourceFormat": source_format, - "sourceUris": source_uris, - "compression": compression, - "ignoreUnknownValues": ignore_unknown_values, - } - - # if following fields are not specified in src_fmt_configs, - # honor the top-level params for backward-compatibility - backward_compatibility_configs = { - "skipLeadingRows": skip_leading_rows, - "fieldDelimiter": field_delimiter, - "quote": quote_character, - "allowQuotedNewlines": allow_quoted_newlines, - "allowJaggedRows": allow_jagged_rows, - "encoding": encoding, - } - src_fmt_to_param_mapping = {"CSV": "csvOptions", "GOOGLE_SHEETS": "googleSheetsOptions"} - src_fmt_to_configs_mapping = { - "csvOptions": [ - "allowJaggedRows", - "allowQuotedNewlines", - "fieldDelimiter", - "skipLeadingRows", - "quote", - "encoding", - ], - "googleSheetsOptions": ["skipLeadingRows"], - } - if source_format in src_fmt_to_param_mapping: - valid_configs = src_fmt_to_configs_mapping[src_fmt_to_param_mapping[source_format]] - src_fmt_configs = _validate_src_fmt_configs( - source_format, src_fmt_configs, valid_configs, backward_compatibility_configs - ) - external_config_api_repr[src_fmt_to_param_mapping[source_format]] = src_fmt_configs - - # build external config - external_config = ExternalConfig.from_api_repr(external_config_api_repr) - if schema_fields: - external_config.schema = [SchemaField.from_api_repr(f) for f in schema_fields] - if max_bad_records: - external_config.max_bad_records = max_bad_records - - # build table definition - table = Table(table_ref=TableReference.from_string(external_project_dataset_table, project_id)) - table.external_data_configuration = external_config - if labels: - table.labels = labels - - if description: - table.description = description - - if encryption_configuration: - table.encryption_configuration = EncryptionConfiguration.from_api_repr(encryption_configuration) - - self.log.info("Creating external table: %s", external_project_dataset_table) - table_object = self.create_empty_table( - table_resource=table.to_api_repr(), project_id=project_id, location=location, exists_ok=True - ) - self.log.info("External table created successfully: %s", external_project_dataset_table) - return table_object - @GoogleBaseHook.fallback_to_default_project_id def update_table( self, @@ -804,113 +618,6 @@ def update_table( self.log.info("Table %s.%s.%s updated successfully", project_id, dataset_id, table_id) return table_object.to_api_repr() - @deprecated( - planned_removal_date="November 01, 2024", - use_instead="airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.update_table", - category=AirflowProviderDeprecationWarning, - ) - @GoogleBaseHook.fallback_to_default_project_id - def patch_table( - self, - dataset_id: str, - table_id: str, - project_id: str = PROVIDE_PROJECT_ID, - description: str | None = None, - expiration_time: int | None = None, - external_data_configuration: dict | None = None, - friendly_name: str | None = None, - labels: dict | None = None, - schema: list | None = None, - time_partitioning: dict | None = None, - view: dict | None = None, - require_partition_filter: bool | None = None, - encryption_configuration: dict | None = None, - ) -> None: - """ - Patch information in an existing table. - - It only updates fields that are provided in the request object. This - method is deprecated. Please use :func:`.update_table` instead. - - Reference: https://cloud.google.com/bigquery/docs/reference/rest/v2/tables/patch - - :param dataset_id: The dataset containing the table to be patched. - :param table_id: The Name of the table to be patched. - :param project_id: The project containing the table to be patched. - :param description: [Optional] A user-friendly description of this table. - :param expiration_time: [Optional] The time when this table expires, - in milliseconds since the epoch. - :param external_data_configuration: [Optional] A dictionary containing - properties of a table stored outside of BigQuery. - :param friendly_name: [Optional] A descriptive name for this table. - :param labels: [Optional] A dictionary containing labels associated with this table. - :param schema: [Optional] If set, the schema field list as defined here: - https://cloud.google.com/bigquery/docs/reference/rest/v2/jobs#configuration.load.schema - The supported schema modifications and unsupported schema modification are listed here: - https://cloud.google.com/bigquery/docs/managing-table-schemas - - .. code-block:: python - - schema = [ - {"name": "emp_name", "type": "STRING", "mode": "REQUIRED"}, - {"name": "salary", "type": "INTEGER", "mode": "NULLABLE"}, - ] - - :param time_partitioning: [Optional] A dictionary containing time-based partitioning - definition for the table. - :param view: [Optional] A dictionary containing definition for the view. - If set, it will patch a view instead of a table: - https://cloud.google.com/bigquery/docs/reference/rest/v2/tables#ViewDefinition - - .. code-block:: python - - view = { - "query": "SELECT * FROM `test-project-id.test_dataset_id.test_table_prefix*` LIMIT 500", - "useLegacySql": False, - } - - :param require_partition_filter: [Optional] If true, queries over the this table require a - partition filter. If false, queries over the table - :param encryption_configuration: [Optional] Custom encryption configuration (e.g., Cloud KMS keys). - - .. code-block:: python - - encryption_configuration = { - "kmsKeyName": "projects/testp/locations/us/keyRings/test-kr/cryptoKeys/test-key", - } - - """ - table_resource: dict[str, Any] = {} - - if description is not None: - table_resource["description"] = description - if expiration_time is not None: - table_resource["expirationTime"] = expiration_time - if external_data_configuration: - table_resource["externalDataConfiguration"] = external_data_configuration - if friendly_name is not None: - table_resource["friendlyName"] = friendly_name - if labels: - table_resource["labels"] = labels - if schema: - table_resource["schema"] = {"fields": schema} - if time_partitioning: - table_resource["timePartitioning"] = time_partitioning - if view: - table_resource["view"] = view - if require_partition_filter is not None: - table_resource["requirePartitionFilter"] = require_partition_filter - if encryption_configuration: - table_resource["encryptionConfiguration"] = encryption_configuration - - self.update_table( - table_resource=table_resource, - fields=list(table_resource.keys()), - project_id=project_id, - dataset_id=dataset_id, - table_id=table_id, - ) - @GoogleBaseHook.fallback_to_default_project_id def insert_all( self, @@ -1014,96 +721,6 @@ def update_dataset( self.log.info("Dataset successfully updated: %s", dataset) return dataset - @deprecated( - planned_removal_date="November 01, 2024", - use_instead="airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.update_dataset", - category=AirflowProviderDeprecationWarning, - ) - def patch_dataset( - self, dataset_id: str, dataset_resource: dict, project_id: str = PROVIDE_PROJECT_ID - ) -> dict: - """ - Patches information in an existing dataset. - - It only replaces fields that are provided in the submitted dataset resource. - - This method is deprecated. Please use :func:`.update_dataset` instead. - - More info: - https://cloud.google.com/bigquery/docs/reference/rest/v2/datasets/patch - - :param dataset_id: The BigQuery Dataset ID - :param dataset_resource: Dataset resource that will be provided - in request body. - https://cloud.google.com/bigquery/docs/reference/rest/v2/datasets#resource - :param project_id: The Google Cloud Project ID - """ - project_id = project_id or self.project_id - if not dataset_id or not isinstance(dataset_id, str): - raise ValueError( - f"dataset_id argument must be provided and has a type 'str'. You provided: {dataset_id}" - ) - - service = self.get_service() - dataset_project_id = project_id or self.project_id - - self.log.info("Start patching dataset: %s:%s", dataset_project_id, dataset_id) - dataset = ( - service.datasets() - .patch( - datasetId=dataset_id, - projectId=dataset_project_id, - body=dataset_resource, - ) - .execute(num_retries=self.num_retries) - ) - self.log.info("Dataset successfully patched: %s", dataset) - - return dataset - - @deprecated( - planned_removal_date="November 01, 2024", - use_instead="airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_dataset_tables", - category=AirflowProviderDeprecationWarning, - ) - def get_dataset_tables_list( - self, - dataset_id: str, - project_id: str = PROVIDE_PROJECT_ID, - table_prefix: str | None = None, - max_results: int | None = None, - ) -> list[dict[str, Any]]: - """ - List tables of a BigQuery dataset. - - If a table prefix is specified, only tables beginning by it are - returned. This method is deprecated. Please use - :func:`.get_dataset_tables` instead. - - For more information, see: - https://cloud.google.com/bigquery/docs/reference/rest/v2/tables/list - - :param dataset_id: The BigQuery Dataset ID - :param project_id: The Google Cloud Project ID - :param table_prefix: Tables must begin by this prefix to be returned (case sensitive) - :param max_results: The maximum number of results to return in a single response page. - Leverage the page tokens to iterate through the entire collection. - :return: List of tables associated with the dataset - """ - project_id = project_id or self.project_id - tables = self.get_client().list_tables( - dataset=DatasetReference(project=project_id, dataset_id=dataset_id), - max_results=max_results, - ) - - if table_prefix: - result = [t.reference.to_api_repr() for t in tables if t.table_id.startswith(table_prefix)] - else: - result = [t.reference.to_api_repr() for t in tables] - - self.log.info("%s tables found", len(result)) - return result - @GoogleBaseHook.fallback_to_default_project_id def get_datasets_list( self, @@ -1266,29 +883,6 @@ def run_table_upsert( ).to_api_repr() return table - @deprecated( - planned_removal_date="November 01, 2024", - use_instead="airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.delete_table", - category=AirflowProviderDeprecationWarning, - ) - def run_table_delete(self, deletion_dataset_table: str, ignore_if_missing: bool = False) -> None: - """ - Delete an existing table from the dataset. - - If the table does not exist, return an error unless *ignore_if_missing* - is set to True. - - This method is deprecated. Please use :func:`.delete_table` instead. - - :param deletion_dataset_table: A dotted - ``(.|:).
`` that indicates which table - will be deleted. - :param ignore_if_missing: if True, then return success even if the - requested table does not exist. - :return: - """ - return self.delete_table(table_id=deletion_dataset_table, not_found_ok=ignore_if_missing) - @GoogleBaseHook.fallback_to_default_project_id def delete_table( self, @@ -1314,47 +908,6 @@ def delete_table( ) self.log.info("Deleted table %s", table_id) - @deprecated( - planned_removal_date="November 01, 2024", - use_instead="airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.list_rows", - category=AirflowProviderDeprecationWarning, - ) - def get_tabledata( - self, - dataset_id: str, - table_id: str, - max_results: int | None = None, - selected_fields: str | None = None, - page_token: str | None = None, - start_index: int | None = None, - ) -> list[dict]: - """ - Get data from given table. - - This method is deprecated. Please use :func:`.list_rows` instead. - - .. seealso:: https://cloud.google.com/bigquery/docs/reference/v2/tabledata/list - - :param dataset_id: the dataset ID of the requested table. - :param table_id: the table ID of the requested table. - :param max_results: the maximum results to return. - :param selected_fields: List of fields to return (comma-separated). If - unspecified, all fields are returned. - :param page_token: page token, returned from a previous call, - identifying the result set. - :param start_index: zero based index of the starting row to read. - :return: list of rows - """ - rows = self.list_rows( - dataset_id=dataset_id, - table_id=table_id, - max_results=max_results, - selected_fields=selected_fields, - page_token=page_token, - start_index=start_index, - ) - return [dict(r) for r in rows] - @GoogleBaseHook.fallback_to_default_project_id def list_rows( self, @@ -1551,18 +1104,6 @@ def poll_job_complete( job = self.get_client(project_id=project_id, location=location).get_job(job_id=job_id) return job.done(retry=retry) - @deprecated( - planned_removal_date="November 01, 2024", - use_instead="airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.cancel_job", - category=AirflowProviderDeprecationWarning, - ) - def cancel_query(self) -> None: - """Cancel all started queries that have not yet completed.""" - if self.running_job_id: - self.cancel_job(job_id=self.running_job_id) - else: - self.log.info("No running BigQuery jobs to cancel.") - @GoogleBaseHook.fallback_to_default_project_id def cancel_job( self, @@ -1705,701 +1246,51 @@ def insert_job( job_api_repr.result(timeout=timeout, retry=retry) return job_api_repr - @deprecated( - planned_removal_date="November 01, 2024", - use_instead="airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.insert_job", - category=AirflowProviderDeprecationWarning, - ) - def run_with_configuration(self, configuration: dict) -> str: - """ - Execute a BigQuery SQL query. - - .. seealso:: https://cloud.google.com/bigquery/docs/reference/v2/jobs - - This method is deprecated. Please use :func:`.insert_job` instead. - - :param configuration: The configuration parameter maps directly to - BigQuery's configuration field in the job object. See - https://cloud.google.com/bigquery/docs/reference/v2/jobs for - details. - """ - job = self.insert_job(configuration=configuration, project_id=self.project_id) - self.running_job_id = job.job_id - return job.job_id - - @deprecated( - planned_removal_date="November 01, 2024", - use_instead="airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.insert_job", - category=AirflowProviderDeprecationWarning, - ) - def run_load( - self, - destination_project_dataset_table: str, - source_uris: list, - schema_fields: list | None = None, - source_format: str = "CSV", - create_disposition: str = "CREATE_IF_NEEDED", - skip_leading_rows: int = 0, - write_disposition: str = "WRITE_EMPTY", - field_delimiter: str = ",", - max_bad_records: int = 0, - quote_character: str | None = None, - ignore_unknown_values: bool = False, - allow_quoted_newlines: bool = False, - allow_jagged_rows: bool = False, - encoding: str = "UTF-8", - schema_update_options: Iterable | None = None, - src_fmt_configs: dict | None = None, - time_partitioning: dict | None = None, - cluster_fields: list | None = None, - autodetect: bool = False, - encryption_configuration: dict | None = None, - labels: dict | None = None, - description: str | None = None, - ) -> str: - """ - Load data from Google Cloud Storage to BigQuery. - - .. seealso:: https://cloud.google.com/bigquery/docs/reference/v2/jobs - - This method is deprecated. Please use :func:`.insert_job` instead. - - :param destination_project_dataset_table: - The dotted ``(.|:).
($)`` BigQuery - table to load data into. If ```` is not included, project will be the - project defined in the connection json. If a partition is specified the - operator will automatically append the data, create a new partition or create - a new DAY partitioned table. - :param schema_fields: The schema field list as defined here: - https://cloud.google.com/bigquery/docs/reference/v2/jobs#configuration.load - Required if autodetect=False; optional if autodetect=True. - :param autodetect: Attempt to autodetect the schema for CSV and JSON - source files. - :param source_uris: The source Google Cloud - Storage URI (e.g. gs://some-bucket/some-file.txt). A single wild - per-object name can be used. - :param source_format: File format to export. - :param create_disposition: The create disposition if the table doesn't exist. - :param skip_leading_rows: Number of rows to skip when loading from a CSV. - :param write_disposition: The write disposition if the table already exists. - :param field_delimiter: The delimiter to use when loading from a CSV. - :param max_bad_records: The maximum number of bad records that BigQuery can - ignore when running the job. - :param quote_character: The value that is used to quote data sections in a CSV - file. - :param ignore_unknown_values: [Optional] Indicates if BigQuery should allow - extra values that are not represented in the table schema. - If true, the extra values are ignored. If false, records with extra columns - are treated as bad records, and if there are too many bad records, an - invalid error is returned in the job result. - :param allow_quoted_newlines: Whether to allow quoted newlines (true) or not - (false). - :param allow_jagged_rows: Accept rows that are missing trailing optional columns. - The missing values are treated as nulls. If false, records with missing - trailing columns are treated as bad records, and if there are too many bad - records, an invalid error is returned in the job result. Only applicable when - source_format is CSV. - :param encoding: The character encoding of the data. + def generate_job_id(self, job_id, dag_id, task_id, logical_date, configuration, force_rerun=False) -> str: + if force_rerun: + hash_base = str(uuid.uuid4()) + else: + hash_base = json.dumps(configuration, sort_keys=True) - .. seealso:: - https://cloud.google.com/bigquery/docs/reference/rest/v2/tables#externalDataConfiguration.csvOptions.encoding - :param schema_update_options: Allows the schema of the destination - table to be updated as a side effect of the load job. - :param src_fmt_configs: configure optional fields specific to the source format - :param time_partitioning: configure optional time partitioning fields i.e. - partition by field, type and expiration as per API specifications. - :param cluster_fields: Request that the result of this load be stored sorted - by one or more columns. BigQuery supports clustering for both partitioned and - non-partitioned tables. The order of columns given determines the sort order. - :param encryption_configuration: [Optional] Custom encryption configuration (e.g., Cloud KMS keys). + uniqueness_suffix = md5(hash_base.encode()).hexdigest() - .. code-block:: python + if job_id: + return f"{job_id}_{uniqueness_suffix}" - encryption_configuration = { - "kmsKeyName": "projects/testp/locations/us/keyRings/test-kr/cryptoKeys/test-key", - } + exec_date = logical_date.isoformat() + job_id = f"airflow_{dag_id}_{task_id}_{exec_date}_{uniqueness_suffix}" + return re.sub(r"[:\-+.]", "_", job_id) - :param labels: A dictionary containing labels for the BiqQuery table. - :param description: A string containing the description for the BigQuery table. - """ - if not self.project_id: - raise ValueError("The project_id should be set") + def split_tablename( + self, table_input: str, default_project_id: str, var_name: str | None = None + ) -> tuple[str, str, str]: + if "." not in table_input: + raise ValueError(f"Expected table name in the format of .
. Got: {table_input}") - # To provide backward compatibility - schema_update_options = list(schema_update_options or []) + if not default_project_id: + raise ValueError("INTERNAL: No default project is specified") - # bigquery only allows certain source formats - # we check to make sure the passed source format is valid - # if it's not, we raise a ValueError - # Refer to this link for more details: - # https://cloud.google.com/bigquery/docs/reference/rest/v2/jobs#configuration.query.tableDefinitions.(key).sourceFormat - - if schema_fields is None and not autodetect: - raise ValueError("You must either pass a schema or autodetect=True.") - - if src_fmt_configs is None: - src_fmt_configs = {} - - source_format = source_format.upper() - allowed_formats = [ - "CSV", - "NEWLINE_DELIMITED_JSON", - "AVRO", - "GOOGLE_SHEETS", - "DATASTORE_BACKUP", - "PARQUET", - ] - if source_format not in allowed_formats: - raise ValueError( - f"{source_format} is not a valid source format. " - f"Please use one of the following types: {allowed_formats}." - ) + def var_print(var_name): + if var_name is None: + return "" + else: + return f"Format exception for {var_name}: " - # bigquery also allows you to define how you want a table's schema to change - # as a side effect of a load - # for more details: - # https://cloud.google.com/bigquery/docs/reference/rest/v2/jobs#configuration.load.schemaUpdateOptions - allowed_schema_update_options = ["ALLOW_FIELD_ADDITION", "ALLOW_FIELD_RELAXATION"] - if not set(allowed_schema_update_options).issuperset(set(schema_update_options)): + if table_input.count(".") + table_input.count(":") > 3: + raise ValueError(f"{var_print(var_name)}Use either : or . to specify project got {table_input}") + cmpt = table_input.rsplit(":", 1) + project_id = None + rest = table_input + if len(cmpt) == 1: + project_id = None + rest = cmpt[0] + elif len(cmpt) == 2 and cmpt[0].count(":") <= 1: + if cmpt[-1].count(".") != 2: + project_id = cmpt[0] + rest = cmpt[1] + else: raise ValueError( - f"{schema_update_options} contains invalid schema update options. " - f"Please only use one or more of the following options: {allowed_schema_update_options}" - ) - - destination_project, destination_dataset, destination_table = self.split_tablename( - table_input=destination_project_dataset_table, - default_project_id=self.project_id, - var_name="destination_project_dataset_table", - ) - - configuration: dict[str, Any] = { - "load": { - "autodetect": autodetect, - "createDisposition": create_disposition, - "destinationTable": { - "projectId": destination_project, - "datasetId": destination_dataset, - "tableId": destination_table, - }, - "sourceFormat": source_format, - "sourceUris": source_uris, - "writeDisposition": write_disposition, - "ignoreUnknownValues": ignore_unknown_values, - } - } - - time_partitioning = _cleanse_time_partitioning(destination_project_dataset_table, time_partitioning) - if time_partitioning: - configuration["load"].update({"timePartitioning": time_partitioning}) - - if cluster_fields: - configuration["load"].update({"clustering": {"fields": cluster_fields}}) - - if schema_fields: - configuration["load"]["schema"] = {"fields": schema_fields} - - if schema_update_options: - if write_disposition not in ["WRITE_APPEND", "WRITE_TRUNCATE"]: - raise ValueError( - "schema_update_options is only " - "allowed if write_disposition is " - "'WRITE_APPEND' or 'WRITE_TRUNCATE'." - ) - else: - self.log.info("Adding experimental 'schemaUpdateOptions': %s", schema_update_options) - configuration["load"]["schemaUpdateOptions"] = schema_update_options - - if max_bad_records: - configuration["load"]["maxBadRecords"] = max_bad_records - - if encryption_configuration: - configuration["load"]["destinationEncryptionConfiguration"] = encryption_configuration - - if labels or description: - configuration["load"].update({"destinationTableProperties": {}}) - - if labels: - configuration["load"]["destinationTableProperties"]["labels"] = labels - - if description: - configuration["load"]["destinationTableProperties"]["description"] = description - - src_fmt_to_configs_mapping = { - "CSV": [ - "allowJaggedRows", - "allowQuotedNewlines", - "autodetect", - "fieldDelimiter", - "skipLeadingRows", - "ignoreUnknownValues", - "nullMarker", - "quote", - "encoding", - "preserveAsciiControlCharacters", - ], - "DATASTORE_BACKUP": ["projectionFields"], - "NEWLINE_DELIMITED_JSON": ["autodetect", "ignoreUnknownValues"], - "PARQUET": ["autodetect", "ignoreUnknownValues"], - "AVRO": ["useAvroLogicalTypes"], - } - - valid_configs = src_fmt_to_configs_mapping[source_format] - - # if following fields are not specified in src_fmt_configs, - # honor the top-level params for backward-compatibility - backward_compatibility_configs = { - "skipLeadingRows": skip_leading_rows, - "fieldDelimiter": field_delimiter, - "ignoreUnknownValues": ignore_unknown_values, - "quote": quote_character, - "allowQuotedNewlines": allow_quoted_newlines, - "encoding": encoding, - } - - src_fmt_configs = _validate_src_fmt_configs( - source_format, src_fmt_configs, valid_configs, backward_compatibility_configs - ) - - configuration["load"].update(src_fmt_configs) - - if allow_jagged_rows: - configuration["load"]["allowJaggedRows"] = allow_jagged_rows - - job = self.insert_job(configuration=configuration, project_id=self.project_id) - self.running_job_id = job.job_id - return job.job_id - - @deprecated( - planned_removal_date="November 01, 2024", - use_instead="airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.insert_job", - category=AirflowProviderDeprecationWarning, - ) - def run_copy( - self, - source_project_dataset_tables: list | str, - destination_project_dataset_table: str, - write_disposition: str = "WRITE_EMPTY", - create_disposition: str = "CREATE_IF_NEEDED", - labels: dict | None = None, - encryption_configuration: dict | None = None, - ) -> str: - """ - Copy data from one BigQuery table to another. - - .. seealso:: https://cloud.google.com/bigquery/docs/reference/v2/jobs#configuration.copy - - This method is deprecated. Please use :func:`.insert_job` instead. - - :param source_project_dataset_tables: One or more dotted - ``(project:|project.).
`` - BigQuery tables to use as the source data. Use a list if there are - multiple source tables. - If ```` is not included, project will be the project defined - in the connection json. - :param destination_project_dataset_table: The destination BigQuery - table. Format is: ``(project:|project.).
`` - :param write_disposition: The write disposition if the table already exists. - :param create_disposition: The create disposition if the table doesn't exist. - :param labels: a dictionary containing labels for the job/query, - passed to BigQuery - :param encryption_configuration: [Optional] Custom encryption configuration (e.g., Cloud KMS keys). - - .. code-block:: python - - encryption_configuration = { - "kmsKeyName": "projects/testp/locations/us/keyRings/test-kr/cryptoKeys/test-key", - } - """ - if not self.project_id: - raise ValueError("The project_id should be set") - - source_project_dataset_tables = ( - [source_project_dataset_tables] - if not isinstance(source_project_dataset_tables, list) - else source_project_dataset_tables - ) - - source_project_dataset_tables_fixup = [] - for source_project_dataset_table in source_project_dataset_tables: - source_project, source_dataset, source_table = self.split_tablename( - table_input=source_project_dataset_table, - default_project_id=self.project_id, - var_name="source_project_dataset_table", - ) - source_project_dataset_tables_fixup.append( - {"projectId": source_project, "datasetId": source_dataset, "tableId": source_table} - ) - - destination_project, destination_dataset, destination_table = self.split_tablename( - table_input=destination_project_dataset_table, default_project_id=self.project_id - ) - configuration = { - "copy": { - "createDisposition": create_disposition, - "writeDisposition": write_disposition, - "sourceTables": source_project_dataset_tables_fixup, - "destinationTable": { - "projectId": destination_project, - "datasetId": destination_dataset, - "tableId": destination_table, - }, - } - } - - if labels: - configuration["labels"] = labels - - if encryption_configuration: - configuration["copy"]["destinationEncryptionConfiguration"] = encryption_configuration - - job = self.insert_job(configuration=configuration, project_id=self.project_id) - self.running_job_id = job.job_id - return job.job_id - - @deprecated( - planned_removal_date="November 01, 2024", - use_instead="airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.insert_job", - category=AirflowProviderDeprecationWarning, - ) - def run_extract( - self, - source_project_dataset_table: str, - destination_cloud_storage_uris: list[str], - compression: str = "NONE", - export_format: str = "CSV", - field_delimiter: str = ",", - print_header: bool = True, - labels: dict | None = None, - return_full_job: bool = False, - ) -> str | BigQueryJob: - """ - Copy data from BigQuery to Google Cloud Storage. - - .. seealso:: https://cloud.google.com/bigquery/docs/reference/v2/jobs - - This method is deprecated. Please use :func:`.insert_job` instead. - - :param source_project_dataset_table: The dotted ``.
`` - BigQuery table to use as the source data. - :param destination_cloud_storage_uris: The destination Google Cloud - Storage URI (e.g. gs://some-bucket/some-file.txt). Follows - convention defined here: - https://cloud.google.com/bigquery/exporting-data-from-bigquery#exportingmultiple - :param compression: Type of compression to use. - :param export_format: File format to export. - :param field_delimiter: The delimiter to use when extracting to a CSV. - :param print_header: Whether to print a header for a CSV file extract. - :param labels: a dictionary containing labels for the job/query, - passed to BigQuery - :param return_full_job: return full job instead of job id only - """ - if not self.project_id: - raise ValueError("The project_id should be set") - - source_project, source_dataset, source_table = self.split_tablename( - table_input=source_project_dataset_table, - default_project_id=self.project_id, - var_name="source_project_dataset_table", - ) - - configuration: dict[str, Any] = { - "extract": { - "sourceTable": { - "projectId": source_project, - "datasetId": source_dataset, - "tableId": source_table, - }, - "compression": compression, - "destinationUris": destination_cloud_storage_uris, - "destinationFormat": export_format, - } - } - - if labels: - configuration["labels"] = labels - - if export_format == "CSV": - # Only set fieldDelimiter and printHeader fields if using CSV. - # Google does not like it if you set these fields for other export - # formats. - configuration["extract"]["fieldDelimiter"] = field_delimiter - configuration["extract"]["printHeader"] = print_header - - job = self.insert_job(configuration=configuration, project_id=self.project_id) - self.running_job_id = job.job_id - if return_full_job: - return job - return job.job_id - - @deprecated( - planned_removal_date="November 01, 2024", - use_instead="airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.insert_job", - category=AirflowProviderDeprecationWarning, - ) - def run_query( - self, - sql: str, - destination_dataset_table: str | None = None, - write_disposition: str = "WRITE_EMPTY", - allow_large_results: bool = False, - flatten_results: bool | None = None, - udf_config: list | None = None, - use_legacy_sql: bool | None = None, - maximum_billing_tier: int | None = None, - maximum_bytes_billed: float | None = None, - create_disposition: str = "CREATE_IF_NEEDED", - query_params: list | None = None, - labels: dict | None = None, - schema_update_options: Iterable | None = None, - priority: str | None = None, - time_partitioning: dict | None = None, - api_resource_configs: dict | None = None, - cluster_fields: list[str] | None = None, - location: str | None = None, - encryption_configuration: dict | None = None, - ) -> str: - """ - Execute a BigQuery SQL query. - - Optionally persists results in a BigQuery table. - - .. seealso:: https://cloud.google.com/bigquery/docs/reference/v2/jobs - - This method is deprecated. Please use :func:`.insert_job` instead. - - For more details about these parameters. - - :param sql: The BigQuery SQL to execute. - :param destination_dataset_table: The dotted ``.
`` - BigQuery table to save the query results. - :param write_disposition: What to do if the table already exists in - BigQuery. - :param allow_large_results: Whether to allow large results. - :param flatten_results: If true and query uses legacy SQL dialect, flattens - all nested and repeated fields in the query results. ``allowLargeResults`` - must be true if this is set to false. For standard SQL queries, this - flag is ignored and results are never flattened. - :param udf_config: The User Defined Function configuration for the query. - See https://cloud.google.com/bigquery/user-defined-functions for details. - :param use_legacy_sql: Whether to use legacy SQL (true) or standard SQL (false). - If `None`, defaults to `self.use_legacy_sql`. - :param api_resource_configs: a dictionary that contain params - 'configuration' applied for Google BigQuery Jobs API: - https://cloud.google.com/bigquery/docs/reference/rest/v2/jobs - for example, {'query': {'useQueryCache': False}}. You could use it - if you need to provide some params that are not supported by the - BigQueryHook like args. - :param maximum_billing_tier: Positive integer that serves as a - multiplier of the basic price. - :param maximum_bytes_billed: Limits the bytes billed for this job. - Queries that will have bytes billed beyond this limit will fail - (without incurring a charge). If unspecified, this will be - set to your project default. - :param create_disposition: Specifies whether the job is allowed to - create new tables. - :param query_params: a list of dictionary containing query parameter types and - values, passed to BigQuery - :param labels: a dictionary containing labels for the job/query, - passed to BigQuery - :param schema_update_options: Allows the schema of the destination - table to be updated as a side effect of the query job. - :param priority: Specifies a priority for the query. - Possible values include INTERACTIVE and BATCH. - If `None`, defaults to `self.priority`. - :param time_partitioning: configure optional time partitioning fields i.e. - partition by field, type and expiration as per API specifications. - :param cluster_fields: Request that the result of this query be stored sorted - by one or more columns. BigQuery supports clustering for both partitioned and - non-partitioned tables. The order of columns given determines the sort order. - :param location: The geographic location of the job. Required except for - US and EU. See details at - https://cloud.google.com/bigquery/docs/locations#specifying_your_location - :param encryption_configuration: [Optional] Custom encryption configuration (e.g., Cloud KMS keys). - - .. code-block:: python - - encryption_configuration = { - "kmsKeyName": "projects/testp/locations/us/keyRings/test-kr/cryptoKeys/test-key", - } - """ - if not self.project_id: - raise ValueError("The project_id should be set") - - labels = labels or self.labels - schema_update_options = list(schema_update_options or []) - - priority = priority or self.priority - - if time_partitioning is None: - time_partitioning = {} - - if not api_resource_configs: - api_resource_configs = self.api_resource_configs - else: - _validate_value("api_resource_configs", api_resource_configs, dict) - configuration = deepcopy(api_resource_configs) - if "query" not in configuration: - configuration["query"] = {} - - else: - _validate_value("api_resource_configs['query']", configuration["query"], dict) - - if sql is None and not configuration["query"].get("query", None): - raise TypeError("`BigQueryBaseCursor.run_query` missing 1 required positional argument: `sql`") - - # BigQuery also allows you to define how you want a table's schema to change - # as a side effect of a query job - # for more details: - # https://cloud.google.com/bigquery/docs/reference/rest/v2/jobs#configuration.query.schemaUpdateOptions - - allowed_schema_update_options = ["ALLOW_FIELD_ADDITION", "ALLOW_FIELD_RELAXATION"] - - if not set(allowed_schema_update_options).issuperset(set(schema_update_options)): - raise ValueError( - f"{schema_update_options} contains invalid schema update options." - f" Please only use one or more of the following options: {allowed_schema_update_options}" - ) - - if schema_update_options: - if write_disposition not in ["WRITE_APPEND", "WRITE_TRUNCATE"]: - raise ValueError( - "schema_update_options is only " - "allowed if write_disposition is " - "'WRITE_APPEND' or 'WRITE_TRUNCATE'." - ) - - if destination_dataset_table: - destination_project, destination_dataset, destination_table = self.split_tablename( - table_input=destination_dataset_table, default_project_id=self.project_id - ) - - destination_dataset_table = { # type: ignore - "projectId": destination_project, - "datasetId": destination_dataset, - "tableId": destination_table, - } - - if cluster_fields: - cluster_fields = {"fields": cluster_fields} # type: ignore - - query_param_list: list[tuple[Any, str, str | bool | None | dict, type | tuple[type]]] = [ - (sql, "query", None, (str,)), - (priority, "priority", priority, (str,)), - (use_legacy_sql, "useLegacySql", self.use_legacy_sql, bool), - (query_params, "queryParameters", None, list), - (udf_config, "userDefinedFunctionResources", None, list), - (maximum_billing_tier, "maximumBillingTier", None, int), - (maximum_bytes_billed, "maximumBytesBilled", None, float), - (time_partitioning, "timePartitioning", {}, dict), - (schema_update_options, "schemaUpdateOptions", None, list), - (destination_dataset_table, "destinationTable", None, dict), - (cluster_fields, "clustering", None, dict), - ] - - for param, param_name, param_default, param_type in query_param_list: - if param_name not in configuration["query"] and param in [None, {}, ()]: - if param_name == "timePartitioning": - param_default = _cleanse_time_partitioning(destination_dataset_table, time_partitioning) - param = param_default - - if param in [None, {}, ()]: - continue - - _api_resource_configs_duplication_check(param_name, param, configuration["query"]) - - configuration["query"][param_name] = param - - # check valid type of provided param, - # it last step because we can get param from 2 sources, - # and first of all need to find it - - _validate_value(param_name, configuration["query"][param_name], param_type) - - if param_name == "schemaUpdateOptions" and param: - self.log.info("Adding experimental 'schemaUpdateOptions': %s", schema_update_options) - - if param_name == "destinationTable": - for key in ["projectId", "datasetId", "tableId"]: - if key not in configuration["query"]["destinationTable"]: - raise ValueError( - "Not correct 'destinationTable' in " - "api_resource_configs. 'destinationTable' " - "must be a dict with {'projectId':'', " - "'datasetId':'', 'tableId':''}" - ) - else: - configuration["query"].update( - { - "allowLargeResults": allow_large_results, - "flattenResults": flatten_results, - "writeDisposition": write_disposition, - "createDisposition": create_disposition, - } - ) - - if ( - "useLegacySql" in configuration["query"] - and configuration["query"]["useLegacySql"] - and "queryParameters" in configuration["query"] - ): - raise ValueError("Query parameters are not allowed when using legacy SQL") - - if labels: - _api_resource_configs_duplication_check("labels", labels, configuration) - configuration["labels"] = labels - - if encryption_configuration: - configuration["query"]["destinationEncryptionConfiguration"] = encryption_configuration - - job = self.insert_job(configuration=configuration, project_id=self.project_id, location=location) - self.running_job_id = job.job_id - return job.job_id - - def generate_job_id(self, job_id, dag_id, task_id, logical_date, configuration, force_rerun=False) -> str: - if force_rerun: - hash_base = str(uuid.uuid4()) - else: - hash_base = json.dumps(configuration, sort_keys=True) - - uniqueness_suffix = md5(hash_base.encode()).hexdigest() - - if job_id: - return f"{job_id}_{uniqueness_suffix}" - - exec_date = logical_date.isoformat() - job_id = f"airflow_{dag_id}_{task_id}_{exec_date}_{uniqueness_suffix}" - return re.sub(r"[:\-+.]", "_", job_id) - - def split_tablename( - self, table_input: str, default_project_id: str, var_name: str | None = None - ) -> tuple[str, str, str]: - if "." not in table_input: - raise ValueError(f"Expected table name in the format of .
. Got: {table_input}") - - if not default_project_id: - raise ValueError("INTERNAL: No default project is specified") - - def var_print(var_name): - if var_name is None: - return "" - else: - return f"Format exception for {var_name}: " - - if table_input.count(".") + table_input.count(":") > 3: - raise ValueError(f"{var_print(var_name)}Use either : or . to specify project got {table_input}") - cmpt = table_input.rsplit(":", 1) - project_id = None - rest = table_input - if len(cmpt) == 1: - project_id = None - rest = cmpt[0] - elif len(cmpt) == 2 and cmpt[0].count(":") <= 1: - if cmpt[-1].count(".") != 2: - project_id = cmpt[0] - rest = cmpt[1] - else: - raise ValueError( - f"{var_print(var_name)}Expect format of (.
, got {table_input}" + f"{var_print(var_name)}Expect format of (.
, got {table_input}" ) cmpt = rest.split(".") @@ -2548,343 +1439,6 @@ def __init__( self.labels = labels self.hook = hook - @deprecated( - planned_removal_date="November 01, 2024", - use_instead="airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.create_empty_table", - category=AirflowProviderDeprecationWarning, - ) - def create_empty_table(self, *args, **kwargs): - """ - Create empty table. DEPRECATED. - - Please use :func:`~airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.create_empty_table` - instead. - """ - return self.hook.create_empty_table(*args, **kwargs) - - @deprecated( - planned_removal_date="November 01, 2024", - use_instead="airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.create_empty_dataset", - category=AirflowProviderDeprecationWarning, - ) - def create_empty_dataset(self, *args, **kwargs) -> dict[str, Any]: - """ - Create empty dataset. DEPRECATED. - - Please use :func:`~airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.create_empty_dataset` - instead. - """ - return self.hook.create_empty_dataset(*args, **kwargs) - - @deprecated( - planned_removal_date="November 01, 2024", - use_instead="airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_dataset_tables", - category=AirflowProviderDeprecationWarning, - ) - def get_dataset_tables(self, *args, **kwargs) -> list[dict[str, Any]]: - """ - Get dataset tables. DEPRECATED. - - Please use :func:`~airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_dataset_tables` - instead. - """ - return self.hook.get_dataset_tables(*args, **kwargs) - - @deprecated( - planned_removal_date="November 01, 2024", - use_instead="airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.delete_dataset", - category=AirflowProviderDeprecationWarning, - ) - def delete_dataset(self, *args, **kwargs) -> None: - """ - Delete dataset. DEPRECATED. - - Please use :func:`~airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.delete_dataset` - instead. - """ - return self.hook.delete_dataset(*args, **kwargs) - - @deprecated( - planned_removal_date="November 01, 2024", - use_instead="airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.create_external_table", - category=AirflowProviderDeprecationWarning, - ) - def create_external_table(self, *args, **kwargs): - """ - Create external table. DEPRECATED. - - Please use :func:`~airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.create_external_table` - instead. - """ - return self.hook.create_external_table(*args, **kwargs) - - @deprecated( - planned_removal_date="November 01, 2024", - use_instead="airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.patch_table", - category=AirflowProviderDeprecationWarning, - ) - def patch_table(self, *args, **kwargs) -> None: - """ - Patch table. DEPRECATED. - - Please use :func:`~airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.patch_table` - instead. - """ - return self.hook.patch_table(*args, **kwargs) - - @deprecated( - planned_removal_date="November 01, 2024", - use_instead="airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.insert_all", - category=AirflowProviderDeprecationWarning, - ) - def insert_all(self, *args, **kwargs) -> None: - """ - Insert all. DEPRECATED. - - Please use :func:`~airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.insert_all` - instead. - """ - return self.hook.insert_all(*args, **kwargs) - - @deprecated( - planned_removal_date="November 01, 2024", - use_instead="airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.update_dataset", - category=AirflowProviderDeprecationWarning, - ) - def update_dataset(self, *args, **kwargs) -> dict: - """ - Update dataset. DEPRECATED. - - Please use :func:`~airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.update_dataset` - instead. - """ - return Dataset.to_api_repr(self.hook.update_dataset(*args, **kwargs)) - - @deprecated( - planned_removal_date="November 01, 2024", - use_instead="airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.patch_dataset", - category=AirflowProviderDeprecationWarning, - ) - def patch_dataset(self, *args, **kwargs) -> dict: - """ - Patch dataset. DEPRECATED. - - Please use :func:`~airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.patch_dataset` - instead. - """ - return self.hook.patch_dataset(*args, **kwargs) - - @deprecated( - planned_removal_date="November 01, 2024", - use_instead="airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_dataset_tables_list", - category=AirflowProviderDeprecationWarning, - ) - def get_dataset_tables_list(self, *args, **kwargs) -> list[dict[str, Any]]: - """ - Get dataset tables list. DEPRECATED. - - Please use :func:`~airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_dataset_tables_list` - instead. - """ - return self.hook.get_dataset_tables_list(*args, **kwargs) - - @deprecated( - planned_removal_date="November 01, 2024", - use_instead="airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_datasets_list", - category=AirflowProviderDeprecationWarning, - ) - def get_datasets_list(self, *args, **kwargs) -> list | HTTPIterator: - """ - Get datasets list. DEPRECATED. - - Please use :func:`~airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_datasets_list` - instead. - """ - return self.hook.get_datasets_list(*args, **kwargs) - - @deprecated( - planned_removal_date="November 01, 2024", - use_instead="airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_dataset", - category=AirflowProviderDeprecationWarning, - ) - def get_dataset(self, *args, **kwargs) -> Dataset: - """ - Get dataset. DEPRECATED. - - Please use :func:`~airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_dataset` - instead. - """ - return self.hook.get_dataset(*args, **kwargs) - - @deprecated( - planned_removal_date="November 01, 2024", - use_instead="airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.run_grant_dataset_view_access", - category=AirflowProviderDeprecationWarning, - ) - def run_grant_dataset_view_access(self, *args, **kwargs) -> dict: - """ - Grant view access to dataset. DEPRECATED. - - Please use - :func:`~airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.run_grant_dataset_view_access` - instead. - """ - return self.hook.run_grant_dataset_view_access(*args, **kwargs) - - @deprecated( - planned_removal_date="November 01, 2024", - use_instead="airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.run_table_upsert", - category=AirflowProviderDeprecationWarning, - ) - def run_table_upsert(self, *args, **kwargs) -> dict: - """ - Upsert table. DEPRECATED. - - Please use :func:`~airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.run_table_upsert` - instead. - """ - return self.hook.run_table_upsert(*args, **kwargs) - - @deprecated( - planned_removal_date="November 01, 2024", - use_instead="airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.run_table_delete", - category=AirflowProviderDeprecationWarning, - ) - def run_table_delete(self, *args, **kwargs) -> None: - """ - Delete table. DEPRECATED. - - Please use :func:`~airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.run_table_delete` - instead. - """ - return self.hook.run_table_delete(*args, **kwargs) - - @deprecated( - planned_removal_date="November 01, 2024", - use_instead="airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_tabledata", - category=AirflowProviderDeprecationWarning, - ) - def get_tabledata(self, *args, **kwargs) -> list[dict]: - """ - Get table data. DEPRECATED. - - Please use :func:`~airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_tabledata` - instead. - """ - return self.hook.get_tabledata(*args, **kwargs) - - @deprecated( - planned_removal_date="November 01, 2024", - use_instead="airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_schema", - category=AirflowProviderDeprecationWarning, - ) - def get_schema(self, *args, **kwargs) -> dict: - """ - Get Schema. DEPRECATED. - - Please use :func:`~airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_schema` - instead. - """ - return self.hook.get_schema(*args, **kwargs) - - @deprecated( - planned_removal_date="November 01, 2024", - use_instead="airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.poll_job_complete", - category=AirflowProviderDeprecationWarning, - ) - def poll_job_complete(self, *args, **kwargs) -> bool: - """ - Poll for job completion.DEPRECATED. - - Please use :func:`~airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.poll_job_complete` - instead. - """ - return self.hook.poll_job_complete(*args, **kwargs) - - @deprecated( - planned_removal_date="November 01, 2024", - use_instead="airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.cancel_query", - category=AirflowProviderDeprecationWarning, - ) - def cancel_query(self, *args, **kwargs) -> None: - """ - Cancel query. DEPRECATED. - - Please use :func:`~airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.cancel_query` - instead. - """ - return self.hook.cancel_query(*args, **kwargs) # type: ignore - - @deprecated( - planned_removal_date="November 01, 2024", - use_instead="airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.run_with_configuration", - category=AirflowProviderDeprecationWarning, - ) - def run_with_configuration(self, *args, **kwargs) -> str: - """ - Run with configuration. DEPRECATED. - - Please use :func:`~airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.run_with_configuration` - instead. - """ - return self.hook.run_with_configuration(*args, **kwargs) - - @deprecated( - planned_removal_date="November 01, 2024", - use_instead="airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.run_load", - category=AirflowProviderDeprecationWarning, - ) - def run_load(self, *args, **kwargs) -> str: - """ - Run load. DEPRECATED. - - Please use :func:`~airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.run_load` - instead. - """ - return self.hook.run_load(*args, **kwargs) - - @deprecated( - planned_removal_date="November 01, 2024", - use_instead="airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.run_copy", - category=AirflowProviderDeprecationWarning, - ) - def run_copy(self, *args, **kwargs) -> str: - """ - Run copy. DEPRECATED. - - Please use :func:`~airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.run_copy` - instead. - """ - return self.hook.run_copy(*args, **kwargs) - - @deprecated( - planned_removal_date="November 01, 2024", - use_instead="airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.run_extract", - category=AirflowProviderDeprecationWarning, - ) - def run_extract(self, *args, **kwargs) -> str | BigQueryJob: - """ - Run extraction. DEPRECATED. - - Please use :func:`~airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.run_extract` - instead. - """ - return self.hook.run_extract(*args, **kwargs) - - @deprecated( - planned_removal_date="November 01, 2024", - use_instead="airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.run_query", - category=AirflowProviderDeprecationWarning, - ) - def run_query(self, *args, **kwargs) -> str: - """ - Run query. DEPRECATED. - - Please use :func:`~airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.run_query` - instead. - """ - return self.hook.run_query(*args, **kwargs) - class BigQueryCursor(BigQueryBaseCursor): """ diff --git a/providers/src/airflow/providers/google/cloud/hooks/bigquery_dts.py b/providers/src/airflow/providers/google/cloud/hooks/bigquery_dts.py index c76956cac877..3ac1bbd95b46 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/bigquery_dts.py +++ b/providers/src/airflow/providers/google/cloud/hooks/bigquery_dts.py @@ -65,14 +65,10 @@ def __init__( impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) super().__init__( gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, + **kwargs, ) self.location = location @@ -291,15 +287,11 @@ def __init__( impersonation_chain: str | Sequence[str] | None = None, **kwargs, ): - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) super().__init__( gcp_conn_id=gcp_conn_id, location=location, impersonation_chain=impersonation_chain, + **kwargs, ) self._conn: DataTransferServiceAsyncClient | None = None diff --git a/providers/src/airflow/providers/google/cloud/hooks/bigtable.py b/providers/src/airflow/providers/google/cloud/hooks/bigtable.py index 706cdfe5502a..5f06e6eee6e7 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/bigtable.py +++ b/providers/src/airflow/providers/google/cloud/hooks/bigtable.py @@ -49,14 +49,10 @@ def __init__( impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) super().__init__( gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, + **kwargs, ) self._client: Client | None = None diff --git a/providers/src/airflow/providers/google/cloud/hooks/cloud_batch.py b/providers/src/airflow/providers/google/cloud/hooks/cloud_batch.py index 795e82e0e272..e2d8dc810841 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/cloud_batch.py +++ b/providers/src/airflow/providers/google/cloud/hooks/cloud_batch.py @@ -60,8 +60,9 @@ def __init__( self, gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: - super().__init__(gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain) + super().__init__(gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, **kwargs) self._client: BatchServiceClient | None = None def get_conn(self): @@ -183,9 +184,10 @@ def __init__( self, gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ): self._client: BatchServiceAsyncClient | None = None - super().__init__(gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain) + super().__init__(gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, **kwargs) def get_conn(self): if self._client is None: diff --git a/providers/src/airflow/providers/google/cloud/hooks/cloud_build.py b/providers/src/airflow/providers/google/cloud/hooks/cloud_build.py index e4531b7f8d2c..fcb83d4bfd2d 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/cloud_build.py +++ b/providers/src/airflow/providers/google/cloud/hooks/cloud_build.py @@ -62,12 +62,7 @@ def __init__( impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) - super().__init__(gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain) + super().__init__(gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, **kwargs) self._client: dict[str, CloudBuildClient] = {} def _get_build_id_from_operation(self, operation: Operation) -> str: @@ -634,14 +629,6 @@ def update_build_trigger( class CloudBuildAsyncHook(GoogleBaseHook): """Asynchronous Hook for the Google Cloud Build Service.""" - def __init__(self, **kwargs): - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) - super().__init__(**kwargs) - @GoogleBaseHook.fallback_to_default_project_id async def get_cloud_build( self, diff --git a/providers/src/airflow/providers/google/cloud/hooks/cloud_composer.py b/providers/src/airflow/providers/google/cloud/hooks/cloud_composer.py index 4ee41c9ffd62..5d659ef81f61 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/cloud_composer.py +++ b/providers/src/airflow/providers/google/cloud/hooks/cloud_composer.py @@ -57,14 +57,6 @@ class CloudComposerHook(GoogleBaseHook): client_options = ClientOptions(api_endpoint="composer.googleapis.com:443") - def __init__(self, **kwargs): - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) - super().__init__(**kwargs) - def get_environment_client(self) -> EnvironmentsClient: """Retrieve client library object that allow access Environments service.""" return EnvironmentsClient( @@ -425,14 +417,6 @@ def wait_command_execution_result( class CloudComposerAsyncHook(GoogleBaseHook): """Hook for Google Cloud Composer async APIs.""" - def __init__(self, **kwargs): - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) - super().__init__(**kwargs) - client_options = ClientOptions(api_endpoint="composer.googleapis.com:443") def get_environment_client(self) -> EnvironmentsAsyncClient: diff --git a/providers/src/airflow/providers/google/cloud/hooks/cloud_memorystore.py b/providers/src/airflow/providers/google/cloud/hooks/cloud_memorystore.py index 40126856704f..768e9b6c62ec 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/cloud_memorystore.py +++ b/providers/src/airflow/providers/google/cloud/hooks/cloud_memorystore.py @@ -76,14 +76,10 @@ def __init__( impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) super().__init__( gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, + **kwargs, ) self._client: CloudRedisClient | None = None @@ -512,10 +508,12 @@ def __init__( self, gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: super().__init__( gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, + **kwargs, ) self._client: CloudMemcacheClient | None = None diff --git a/providers/src/airflow/providers/google/cloud/hooks/cloud_run.py b/providers/src/airflow/providers/google/cloud/hooks/cloud_run.py index 5eee37973aa9..f50cb59646d6 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/cloud_run.py +++ b/providers/src/airflow/providers/google/cloud/hooks/cloud_run.py @@ -68,8 +68,9 @@ def __init__( self, gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: - super().__init__(gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain) + super().__init__(gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, **kwargs) self._client: JobsClient | None = None def get_conn(self): @@ -176,9 +177,10 @@ def __init__( self, gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ): self._client: JobsAsyncClient | None = None - super().__init__(gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain) + super().__init__(gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, **kwargs) def get_conn(self): if self._client is None: @@ -211,9 +213,10 @@ def __init__( self, gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ): self._client: ServicesClient | None = None - super().__init__(gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain) + super().__init__(gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, **kwargs) def get_conn(self): if self._client is None: @@ -273,9 +276,10 @@ def __init__( self, gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ): self._client: ServicesClient | None = None - super().__init__(gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain) + super().__init__(gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, **kwargs) def get_conn(self): if self._client is None: diff --git a/providers/src/airflow/providers/google/cloud/hooks/cloud_sql.py b/providers/src/airflow/providers/google/cloud/hooks/cloud_sql.py index 9db7db5a1ff4..04bf84d90972 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/cloud_sql.py +++ b/providers/src/airflow/providers/google/cloud/hooks/cloud_sql.py @@ -108,14 +108,10 @@ def __init__( impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) super().__init__( gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, + **kwargs, ) self.api_version = api_version self._conn = None @@ -828,8 +824,9 @@ def __init__( ssl_key: str | None = None, ssl_root_cert: str | None = None, ssl_secret_id: str | None = None, + **kwargs, ) -> None: - super().__init__() + super().__init__(**kwargs) self.gcp_conn_id = gcp_conn_id self.gcp_cloudsql_conn_id = gcp_cloudsql_conn_id self.impersonation_chain = impersonation_chain diff --git a/providers/src/airflow/providers/google/cloud/hooks/cloud_storage_transfer_service.py b/providers/src/airflow/providers/google/cloud/hooks/cloud_storage_transfer_service.py index 607090ff6ae1..215cb717b449 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/cloud_storage_transfer_service.py +++ b/providers/src/airflow/providers/google/cloud/hooks/cloud_storage_transfer_service.py @@ -159,15 +159,10 @@ def __init__( impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: - if "delegate_to" in kwargs: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and " - "finally removed in this version of Google Provider. You MUST " - "convert it to `impersonate_chain`." - ) super().__init__( gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, + **kwargs, ) self.api_version = api_version self._conn = None diff --git a/providers/src/airflow/providers/google/cloud/hooks/compute.py b/providers/src/airflow/providers/google/cloud/hooks/compute.py index de31c20c3b21..8882efad58b0 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/compute.py +++ b/providers/src/airflow/providers/google/cloud/hooks/compute.py @@ -61,14 +61,10 @@ def __init__( impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) super().__init__( gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, + **kwargs, ) self.api_version = api_version diff --git a/providers/src/airflow/providers/google/cloud/hooks/compute_ssh.py b/providers/src/airflow/providers/google/cloud/hooks/compute_ssh.py index fc0194429605..c4aaa45e3d68 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/compute_ssh.py +++ b/providers/src/airflow/providers/google/cloud/hooks/compute_ssh.py @@ -121,11 +121,6 @@ def __init__( impersonation_chain: str | None = None, **kwargs, ) -> None: - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonation_chain`" - ) # Ignore original constructor # super().__init__() self.gcp_conn_id = gcp_conn_id diff --git a/providers/src/airflow/providers/google/cloud/hooks/datacatalog.py b/providers/src/airflow/providers/google/cloud/hooks/datacatalog.py index eeb02cdeb971..57418767fd0e 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/datacatalog.py +++ b/providers/src/airflow/providers/google/cloud/hooks/datacatalog.py @@ -61,14 +61,10 @@ def __init__( impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) super().__init__( gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, + **kwargs, ) self._client: DataCatalogClient | None = None diff --git a/providers/src/airflow/providers/google/cloud/hooks/dataflow.py b/providers/src/airflow/providers/google/cloud/hooks/dataflow.py index 9e94990f550f..09231cb099c7 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/dataflow.py +++ b/providers/src/airflow/providers/google/cloud/hooks/dataflow.py @@ -562,11 +562,6 @@ def __init__( expected_terminal_state: str | None = None, **kwargs, ) -> None: - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) self.poll_sleep = poll_sleep self.drain_pipeline = drain_pipeline self.cancel_timeout = cancel_timeout @@ -577,6 +572,7 @@ def __init__( super().__init__( gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, + **kwargs, ) def get_conn(self) -> Resource: @@ -1565,14 +1561,6 @@ class AsyncDataflowHook(GoogleBaseAsyncHook): sync_hook_class = DataflowHook - def __init__(self, **kwargs): - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) - super().__init__(**kwargs) - async def initialize_client(self, client_class): """ Initialize object of the given class. diff --git a/providers/src/airflow/providers/google/cloud/hooks/dataform.py b/providers/src/airflow/providers/google/cloud/hooks/dataform.py index f9a62cd2cd1a..5ca44ade940c 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/dataform.py +++ b/providers/src/airflow/providers/google/cloud/hooks/dataform.py @@ -41,14 +41,6 @@ class DataformHook(GoogleBaseHook): """Hook for Google Cloud DataForm APIs.""" - def __init__(self, **kwargs): - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) - super().__init__(**kwargs) - def get_dataform_client(self) -> DataformClient: """Retrieve client library object that allow access to Cloud Dataform service.""" return DataformClient(credentials=self.get_credentials()) diff --git a/providers/src/airflow/providers/google/cloud/hooks/datafusion.py b/providers/src/airflow/providers/google/cloud/hooks/datafusion.py index 227c4488947c..a4fce57b2e3c 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/datafusion.py +++ b/providers/src/airflow/providers/google/cloud/hooks/datafusion.py @@ -78,14 +78,10 @@ def __init__( impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) super().__init__( gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, + **kwargs, ) self.api_version = api_version @@ -554,14 +550,6 @@ class DataFusionAsyncHook(GoogleBaseAsyncHook): sync_hook_class = DataFusionHook scopes = ["https://www.googleapis.com/auth/cloud-platform"] - def __init__(self, **kwargs): - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) - super().__init__(**kwargs) - @staticmethod def _base_url(instance_url: str, namespace: str) -> str: return urljoin(f"{instance_url}/", f"v3/namespaces/{quote(namespace)}/apps/") diff --git a/providers/src/airflow/providers/google/cloud/hooks/dataplex.py b/providers/src/airflow/providers/google/cloud/hooks/dataplex.py index ff8cf56a5c14..210d9bc75403 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/dataplex.py +++ b/providers/src/airflow/providers/google/cloud/hooks/dataplex.py @@ -85,14 +85,10 @@ def __init__( location: str | None = None, **kwargs, ) -> None: - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) super().__init__( gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, + **kwargs, ) self.api_version = api_version self.location = location @@ -883,7 +879,7 @@ def __init__( impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: - super().__init__(gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain) + super().__init__(gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, **kwargs) async def get_dataplex_data_scan_client(self) -> DataScanServiceAsyncClient: """Return DataScanServiceAsyncClient.""" diff --git a/providers/src/airflow/providers/google/cloud/hooks/dataprep.py b/providers/src/airflow/providers/google/cloud/hooks/dataprep.py index ee6d55e0903d..6c2db920c637 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/dataprep.py +++ b/providers/src/airflow/providers/google/cloud/hooks/dataprep.py @@ -73,8 +73,8 @@ class GoogleDataprepHook(BaseHook): conn_type = "dataprep" hook_name = "Google Dataprep" - def __init__(self, dataprep_conn_id: str = default_conn_name, api_version: str = "v4") -> None: - super().__init__() + def __init__(self, dataprep_conn_id: str = default_conn_name, api_version: str = "v4", **kwargs) -> None: + super().__init__(**kwargs) self.dataprep_conn_id = dataprep_conn_id self.api_version = api_version conn = self.get_connection(self.dataprep_conn_id) diff --git a/providers/src/airflow/providers/google/cloud/hooks/dataproc.py b/providers/src/airflow/providers/google/cloud/hooks/dataproc.py index 456fd13858f5..f8bedec9853e 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/dataproc.py +++ b/providers/src/airflow/providers/google/cloud/hooks/dataproc.py @@ -217,19 +217,6 @@ class DataprocHook(GoogleBaseHook): keyword arguments rather than positional. """ - def __init__( - self, - gcp_conn_id: str = "google_cloud_default", - impersonation_chain: str | Sequence[str] | None = None, - **kwargs, - ) -> None: - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) - super().__init__(gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain) - def get_cluster_client(self, region: str | None = None) -> ClusterControllerClient: """Create a ClusterControllerClient.""" client_options = None @@ -1219,12 +1206,7 @@ def __init__( impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) - super().__init__(gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain) + super().__init__(gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, **kwargs) self._cached_client: JobControllerAsyncClient | None = None def get_cluster_client(self, region: str | None = None) -> ClusterControllerAsyncClient: diff --git a/providers/src/airflow/providers/google/cloud/hooks/dataproc_metastore.py b/providers/src/airflow/providers/google/cloud/hooks/dataproc_metastore.py index 9eff41240bab..9700a1e94973 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/dataproc_metastore.py +++ b/providers/src/airflow/providers/google/cloud/hooks/dataproc_metastore.py @@ -40,14 +40,6 @@ class DataprocMetastoreHook(GoogleBaseHook): """Hook for Google Cloud Dataproc Metastore APIs.""" - def __init__(self, **kwargs): - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) - super().__init__(**kwargs) - def get_dataproc_metastore_client(self) -> DataprocMetastoreClient: """Return DataprocMetastoreClient.""" client_options = ClientOptions(api_endpoint="metastore.googleapis.com:443") diff --git a/providers/src/airflow/providers/google/cloud/hooks/datastore.py b/providers/src/airflow/providers/google/cloud/hooks/datastore.py index 345315c4532e..864fa4583575 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/datastore.py +++ b/providers/src/airflow/providers/google/cloud/hooks/datastore.py @@ -44,14 +44,10 @@ def __init__( impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) super().__init__( gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, + **kwargs, ) self.connection = None self.api_version = api_version diff --git a/providers/src/airflow/providers/google/cloud/hooks/dlp.py b/providers/src/airflow/providers/google/cloud/hooks/dlp.py index f9e51e1afd27..b5052ff8090b 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/dlp.py +++ b/providers/src/airflow/providers/google/cloud/hooks/dlp.py @@ -91,14 +91,10 @@ def __init__( impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) super().__init__( gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, + **kwargs, ) self._client: DlpServiceClient | None = None diff --git a/providers/src/airflow/providers/google/cloud/hooks/functions.py b/providers/src/airflow/providers/google/cloud/hooks/functions.py index 3aa99eb93085..d488f1d25049 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/functions.py +++ b/providers/src/airflow/providers/google/cloud/hooks/functions.py @@ -49,14 +49,10 @@ def __init__( impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) super().__init__( gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, + **kwargs, ) self.api_version = api_version diff --git a/providers/src/airflow/providers/google/cloud/hooks/gcs.py b/providers/src/airflow/providers/google/cloud/hooks/gcs.py index 995418f18348..22feb2854ec3 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/gcs.py +++ b/providers/src/airflow/providers/google/cloud/hooks/gcs.py @@ -151,23 +151,6 @@ class GCSHook(GoogleBaseHook): _conn: storage.Client | None = None - def __init__( - self, - gcp_conn_id: str = "google_cloud_default", - impersonation_chain: str | Sequence[str] | None = None, - **kwargs, - ) -> None: - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) - - super().__init__( - gcp_conn_id=gcp_conn_id, - impersonation_chain=impersonation_chain, - ) - def get_conn(self) -> storage.Client: """Return a Google Cloud Storage service object.""" if not self._conn: diff --git a/providers/src/airflow/providers/google/cloud/hooks/gdm.py b/providers/src/airflow/providers/google/cloud/hooks/gdm.py index e8311bba95e1..bcf15152d937 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/gdm.py +++ b/providers/src/airflow/providers/google/cloud/hooks/gdm.py @@ -17,7 +17,7 @@ # under the License. from __future__ import annotations -from typing import Any, Sequence +from typing import Any from googleapiclient.discovery import Resource, build @@ -32,22 +32,6 @@ class GoogleDeploymentManagerHook(GoogleBaseHook): This allows for scheduled and programmatic inspection and deletion of resources managed by GDM. """ - def __init__( - self, - gcp_conn_id: str = "google_cloud_default", - impersonation_chain: str | Sequence[str] | None = None, - **kwargs, - ) -> None: - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) - super().__init__( - gcp_conn_id=gcp_conn_id, - impersonation_chain=impersonation_chain, - ) - def get_conn(self) -> Resource: """Return a Google Deployment Manager service object.""" http_authorized = self._authorize() diff --git a/providers/src/airflow/providers/google/cloud/hooks/kms.py b/providers/src/airflow/providers/google/cloud/hooks/kms.py index 41a1ad64ab8b..5071fc5ffb18 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/kms.py +++ b/providers/src/airflow/providers/google/cloud/hooks/kms.py @@ -63,14 +63,10 @@ def __init__( impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) super().__init__( gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, + **kwargs, ) self._conn: KeyManagementServiceClient | None = None diff --git a/providers/src/airflow/providers/google/cloud/hooks/kubernetes_engine.py b/providers/src/airflow/providers/google/cloud/hooks/kubernetes_engine.py index c8af049dc01e..ef9d79793774 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/kubernetes_engine.py +++ b/providers/src/airflow/providers/google/cloud/hooks/kubernetes_engine.py @@ -29,7 +29,7 @@ from google.auth.transport import requests as google_requests # not sure why but mypy complains on missing `container_v1` but it is clearly there and is importable -from google.cloud import container_v1, exceptions # type: ignore[attr-defined] +from google.cloud import exceptions # type: ignore[attr-defined] from google.cloud.container_v1 import ClusterManagerAsyncClient, ClusterManagerClient from google.cloud.container_v1.types import Cluster, Operation from kubernetes import client, utils @@ -38,11 +38,10 @@ from kubernetes_asyncio.config.kube_config import FileOrData from airflow import version -from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning +from airflow.exceptions import AirflowException from airflow.providers.cncf.kubernetes.hooks.kubernetes import AsyncKubernetesHook, KubernetesHook from airflow.providers.cncf.kubernetes.kube_client import _enable_tcp_keepalive from airflow.providers.google.common.consts import CLIENT_INFO -from airflow.providers.google.common.deprecated import deprecated from airflow.providers.google.common.hooks.base_google import ( PROVIDE_PROJECT_ID, GoogleBaseAsyncHook, @@ -118,14 +117,10 @@ def __init__( impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) super().__init__( gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, + **kwargs, ) self._client: ClusterManagerClient | None = None self.location = location @@ -136,26 +131,6 @@ def get_cluster_manager_client(self) -> ClusterManagerClient: self._client = ClusterManagerClient(credentials=self.get_credentials(), client_info=CLIENT_INFO) return self._client - # To preserve backward compatibility - # TODO: remove one day - @deprecated( - planned_removal_date="November 01, 2024", - use_instead="get_cluster_manager_client", - category=AirflowProviderDeprecationWarning, - ) - def get_conn(self) -> container_v1.ClusterManagerClient: - return self.get_cluster_manager_client() - - # To preserve backward compatibility - # TODO: remove one day - @deprecated( - planned_removal_date="November 01, 2024", - use_instead="get_cluster_manager_client", - category=AirflowProviderDeprecationWarning, - ) - def get_client(self) -> ClusterManagerClient: - return self.get_conn() - def wait_for_operation(self, operation: Operation, project_id: str = PROVIDE_PROJECT_ID) -> Operation: """ Continuously fetch the status from Google Cloud. @@ -388,10 +363,12 @@ def __init__( gcp_conn_id: str = "google_cloud_default", location: str | None = None, impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: super().__init__( gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, + **kwargs, ) self._client: ClusterManagerAsyncClient | None = None self.location = location @@ -576,71 +553,3 @@ def _get_config(self) -> async_client.configuration.Configuration: ).as_file(), ) return configuration - - -@deprecated( - planned_removal_date="October 01, 2024", - use_instead="GKEKubernetesHook", - category=AirflowProviderDeprecationWarning, -) -class GKEDeploymentHook(GKEKubernetesHook): - """Google Kubernetes Engine Deployment APIs.""" - - -@deprecated( - planned_removal_date="October 01, 2024", - use_instead="GKEKubernetesHook", - category=AirflowProviderDeprecationWarning, -) -class GKECustomResourceHook(GKEKubernetesHook): - """Google Kubernetes Engine Custom Resource APIs.""" - - -@deprecated( - planned_removal_date="October 01, 2024", - use_instead="GKEKubernetesHook", - category=AirflowProviderDeprecationWarning, -) -class GKEPodHook(GKEKubernetesHook): - """Google Kubernetes Engine pod APIs.""" - - def __init__( - self, - cluster_url: str, - ssl_ca_cert: str, - disable_tcp_keepalive: bool | None = None, - gcp_conn_id: str = "google_cloud_default", - impersonation_chain: str | Sequence[str] | None = None, - **kwargs, - ): - super().__init__( - gcp_conn_id=gcp_conn_id, - impersonation_chain=impersonation_chain, - cluster_url=cluster_url, - ssl_ca_cert=ssl_ca_cert, - **kwargs, - ) - self.enable_tcp_keepalive = not bool(disable_tcp_keepalive) - - -@deprecated( - planned_removal_date="October 01, 2024", - use_instead="GKEKubernetesHook", - category=AirflowProviderDeprecationWarning, -) -class GKEJobHook(GKEKubernetesHook): - """Google Kubernetes Engine Job APIs.""" - - -@deprecated( - planned_removal_date="October 01, 2024", - use_instead="GKEKubernetesAsyncHook", - category=AirflowProviderDeprecationWarning, -) -class GKEPodAsyncHook(GKEKubernetesAsyncHook): - """ - Google Kubernetes Engine pods APIs asynchronously. - - :param cluster_url: The URL pointed to the cluster. - :param ssl_ca_cert: SSL certificate used for authentication to the pod. - """ diff --git a/providers/src/airflow/providers/google/cloud/hooks/life_sciences.py b/providers/src/airflow/providers/google/cloud/hooks/life_sciences.py index e46a4cebae99..9cf052045d08 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/life_sciences.py +++ b/providers/src/airflow/providers/google/cloud/hooks/life_sciences.py @@ -73,14 +73,10 @@ def __init__( impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) super().__init__( gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, + **kwargs, ) self.api_version = api_version diff --git a/providers/src/airflow/providers/google/cloud/hooks/looker.py b/providers/src/airflow/providers/google/cloud/hooks/looker.py index 80d5a8ef6409..91af73698817 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/looker.py +++ b/providers/src/airflow/providers/google/cloud/hooks/looker.py @@ -42,8 +42,9 @@ class LookerHook(BaseHook): def __init__( self, looker_conn_id: str, + **kwargs, ) -> None: - super().__init__() + super().__init__(**kwargs) self.looker_conn_id = looker_conn_id # source is used to track origin of the requests self.source = f"airflow:{version}" diff --git a/providers/src/airflow/providers/google/cloud/hooks/mlengine.py b/providers/src/airflow/providers/google/cloud/hooks/mlengine.py index 35aa2cb5cd6a..ff33946cf9cb 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/mlengine.py +++ b/providers/src/airflow/providers/google/cloud/hooks/mlengine.py @@ -92,14 +92,6 @@ class MLEngineHook(GoogleBaseHook): keyword arguments rather than positional. """ - def __init__(self, **kwargs): - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) - super().__init__(**kwargs) - def get_conn(self) -> Resource: """ Retrieve the connection to MLEngine. diff --git a/providers/src/airflow/providers/google/cloud/hooks/natural_language.py b/providers/src/airflow/providers/google/cloud/hooks/natural_language.py index b4fbba5bebfd..48c218c15b6b 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/natural_language.py +++ b/providers/src/airflow/providers/google/cloud/hooks/natural_language.py @@ -62,14 +62,10 @@ def __init__( impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) super().__init__( gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, + **kwargs, ) self._conn: LanguageServiceClient | None = None diff --git a/providers/src/airflow/providers/google/cloud/hooks/os_login.py b/providers/src/airflow/providers/google/cloud/hooks/os_login.py index 3b103eb206e5..0829eeff9b97 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/os_login.py +++ b/providers/src/airflow/providers/google/cloud/hooks/os_login.py @@ -50,14 +50,10 @@ def __init__( impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) super().__init__( gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, + **kwargs, ) self._conn: OsLoginServiceClient | None = None diff --git a/providers/src/airflow/providers/google/cloud/hooks/pubsub.py b/providers/src/airflow/providers/google/cloud/hooks/pubsub.py index d8da265a44b9..e2ca684baab3 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/pubsub.py +++ b/providers/src/airflow/providers/google/cloud/hooks/pubsub.py @@ -83,14 +83,10 @@ def __init__( enable_message_ordering: bool = False, **kwargs, ) -> None: - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) super().__init__( gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, + **kwargs, ) self.enable_message_ordering = enable_message_ordering self._client = None diff --git a/providers/src/airflow/providers/google/cloud/hooks/secret_manager.py b/providers/src/airflow/providers/google/cloud/hooks/secret_manager.py index 69c90a3034a3..48983fa70342 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/secret_manager.py +++ b/providers/src/airflow/providers/google/cloud/hooks/secret_manager.py @@ -31,85 +31,14 @@ SecretVersion, ) -from airflow.exceptions import AirflowProviderDeprecationWarning -from airflow.providers.google.cloud._internal_client.secret_manager_client import _SecretManagerClient from airflow.providers.google.common.consts import CLIENT_INFO -from airflow.providers.google.common.deprecated import deprecated -from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID, GoogleBaseHook +from airflow.providers.google.common.hooks.base_google import GoogleBaseHook if TYPE_CHECKING: from google.api_core.retry import Retry from google.cloud.secretmanager_v1.services.secret_manager_service.pagers import ListSecretsPager -@deprecated( - planned_removal_date="November 01, 2024", - use_instead="GoogleCloudSecretManagerHook", - category=AirflowProviderDeprecationWarning, -) -class SecretsManagerHook(GoogleBaseHook): - """ - Hook for the Google Secret Manager API. - - See https://cloud.google.com/secret-manager - - All the methods in the hook where project_id is used must be called with - keyword arguments rather than positional. - - :param gcp_conn_id: The connection ID to use when fetching connection info. - :param impersonation_chain: Optional service account to impersonate using short-term - credentials, or chained list of accounts required to get the access_token - of the last account in the list, which will be impersonated in the request. - If set as a string, the account must grant the originating account - the Service Account Token Creator IAM role. - If set as a sequence, the identities from the list must grant - Service Account Token Creator IAM role to the directly preceding identity, with first - account from the list granting this role to the originating account. - """ - - def __init__( - self, - gcp_conn_id: str = "google_cloud_default", - impersonation_chain: str | Sequence[str] | None = None, - **kwargs, - ) -> None: - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) - super().__init__( - gcp_conn_id=gcp_conn_id, - impersonation_chain=impersonation_chain, - ) - self.client = _SecretManagerClient(credentials=self.get_credentials()) - - def get_conn(self) -> _SecretManagerClient: - """ - Retrieve the connection to Secret Manager. - - :return: Secret Manager client. - """ - return self.client - - @GoogleBaseHook.fallback_to_default_project_id - def get_secret( - self, secret_id: str, secret_version: str = "latest", project_id: str = PROVIDE_PROJECT_ID - ) -> str | None: - """ - Get secret value from the Secret Manager. - - :param secret_id: Secret Key - :param secret_version: version of the secret (default is 'latest') - :param project_id: Project id (if you want to override the project_id from credentials) - """ - return self.get_conn().get_secret( - secret_id=secret_id, - secret_version=secret_version, - project_id=project_id, # type: ignore - ) - - class GoogleCloudSecretManagerHook(GoogleBaseHook): """ Hook for the Google Cloud Secret Manager API. diff --git a/providers/src/airflow/providers/google/cloud/hooks/spanner.py b/providers/src/airflow/providers/google/cloud/hooks/spanner.py index 71715fba99d0..f765e274f596 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/spanner.py +++ b/providers/src/airflow/providers/google/cloud/hooks/spanner.py @@ -64,14 +64,10 @@ def __init__( impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) super().__init__( gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, + **kwargs, ) self._client: Client | None = None diff --git a/providers/src/airflow/providers/google/cloud/hooks/speech_to_text.py b/providers/src/airflow/providers/google/cloud/hooks/speech_to_text.py index e346dfae65eb..4f8f4ddfd5ed 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/speech_to_text.py +++ b/providers/src/airflow/providers/google/cloud/hooks/speech_to_text.py @@ -53,14 +53,10 @@ def __init__( impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) super().__init__( gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, + **kwargs, ) self._client: SpeechClient | None = None diff --git a/providers/src/airflow/providers/google/cloud/hooks/stackdriver.py b/providers/src/airflow/providers/google/cloud/hooks/stackdriver.py index 751e978ab0e2..57fadda62782 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/stackdriver.py +++ b/providers/src/airflow/providers/google/cloud/hooks/stackdriver.py @@ -46,14 +46,10 @@ def __init__( impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) super().__init__( gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, + **kwargs, ) self._policy_client = None self._channel_client = None diff --git a/providers/src/airflow/providers/google/cloud/hooks/tasks.py b/providers/src/airflow/providers/google/cloud/hooks/tasks.py index a7005be9757c..92943482da54 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/tasks.py +++ b/providers/src/airflow/providers/google/cloud/hooks/tasks.py @@ -60,14 +60,10 @@ def __init__( impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) super().__init__( gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, + **kwargs, ) self._client: CloudTasksClient | None = None diff --git a/providers/src/airflow/providers/google/cloud/hooks/text_to_speech.py b/providers/src/airflow/providers/google/cloud/hooks/text_to_speech.py index ce7ead17bbc3..789892cfb4ef 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/text_to_speech.py +++ b/providers/src/airflow/providers/google/cloud/hooks/text_to_speech.py @@ -61,14 +61,10 @@ def __init__( impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) super().__init__( gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, + **kwargs, ) self._client: TextToSpeechClient | None = None diff --git a/providers/src/airflow/providers/google/cloud/hooks/translate.py b/providers/src/airflow/providers/google/cloud/hooks/translate.py index 21c74dc5188b..04a309678a61 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/translate.py +++ b/providers/src/airflow/providers/google/cloud/hooks/translate.py @@ -41,14 +41,10 @@ def __init__( impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) super().__init__( gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, + **kwargs, ) self._client: Client | None = None diff --git a/providers/src/airflow/providers/google/cloud/hooks/vertex_ai/auto_ml.py b/providers/src/airflow/providers/google/cloud/hooks/vertex_ai/auto_ml.py index 3a035022508e..07fabf86e245 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/vertex_ai/auto_ml.py +++ b/providers/src/airflow/providers/google/cloud/hooks/vertex_ai/auto_ml.py @@ -55,14 +55,10 @@ def __init__( impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) super().__init__( gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, + **kwargs, ) self._job: None | ( AutoMLForecastingTrainingJob diff --git a/providers/src/airflow/providers/google/cloud/hooks/vertex_ai/batch_prediction_job.py b/providers/src/airflow/providers/google/cloud/hooks/vertex_ai/batch_prediction_job.py index 8adac65b05e4..7a805dfdf6ee 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/vertex_ai/batch_prediction_job.py +++ b/providers/src/airflow/providers/google/cloud/hooks/vertex_ai/batch_prediction_job.py @@ -46,14 +46,10 @@ def __init__( impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) super().__init__( gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, + **kwargs, ) self._batch_prediction_job: BatchPredictionJob | None = None diff --git a/providers/src/airflow/providers/google/cloud/hooks/vertex_ai/custom_job.py b/providers/src/airflow/providers/google/cloud/hooks/vertex_ai/custom_job.py index 9a9674c201b1..64e9370fc967 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/vertex_ai/custom_job.py +++ b/providers/src/airflow/providers/google/cloud/hooks/vertex_ai/custom_job.py @@ -67,14 +67,10 @@ def __init__( impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) super().__init__( gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, + **kwargs, ) self._job: None | ( CustomContainerTrainingJob | CustomPythonPackageTrainingJob | CustomTrainingJob diff --git a/providers/src/airflow/providers/google/cloud/hooks/vertex_ai/dataset.py b/providers/src/airflow/providers/google/cloud/hooks/vertex_ai/dataset.py index 75ce0fd88663..c3c13d349f79 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/vertex_ai/dataset.py +++ b/providers/src/airflow/providers/google/cloud/hooks/vertex_ai/dataset.py @@ -44,14 +44,6 @@ class DatasetHook(GoogleBaseHook): """Hook for Google Cloud Vertex AI Dataset APIs.""" - def __init__(self, **kwargs): - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) - super().__init__(**kwargs) - def get_dataset_service_client(self, region: str | None = None) -> DatasetServiceClient: """Return DatasetServiceClient.""" if region and region != "global": diff --git a/providers/src/airflow/providers/google/cloud/hooks/vertex_ai/endpoint_service.py b/providers/src/airflow/providers/google/cloud/hooks/vertex_ai/endpoint_service.py index caa78a9a0f4f..ce82089bbfa9 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/vertex_ai/endpoint_service.py +++ b/providers/src/airflow/providers/google/cloud/hooks/vertex_ai/endpoint_service.py @@ -39,14 +39,6 @@ class EndpointServiceHook(GoogleBaseHook): """Hook for Google Cloud Vertex AI Endpoint Service APIs.""" - def __init__(self, **kwargs): - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) - super().__init__(**kwargs) - def get_endpoint_service_client(self, region: str | None = None) -> EndpointServiceClient: """Return EndpointServiceClient.""" if region and region != "global": diff --git a/providers/src/airflow/providers/google/cloud/hooks/vertex_ai/generative_model.py b/providers/src/airflow/providers/google/cloud/hooks/vertex_ai/generative_model.py index 27037baaafda..931cc1927378 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/vertex_ai/generative_model.py +++ b/providers/src/airflow/providers/google/cloud/hooks/vertex_ai/generative_model.py @@ -21,7 +21,7 @@ import time from datetime import timedelta -from typing import TYPE_CHECKING, Sequence +from typing import TYPE_CHECKING import vertexai from vertexai.generative_models import GenerativeModel, Part @@ -43,19 +43,6 @@ class GenerativeModelHook(GoogleBaseHook): """Hook for Google Cloud Vertex AI Generative Model APIs.""" - def __init__( - self, - gcp_conn_id: str = "google_cloud_default", - impersonation_chain: str | Sequence[str] | None = None, - **kwargs, - ): - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) - super().__init__(gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, **kwargs) - def get_text_generation_model(self, pretrained_model: str): """Return a Model Garden Model object based on Text Generation.""" model = TextGenerationModel.from_pretrained(pretrained_model) diff --git a/providers/src/airflow/providers/google/cloud/hooks/vertex_ai/hyperparameter_tuning_job.py b/providers/src/airflow/providers/google/cloud/hooks/vertex_ai/hyperparameter_tuning_job.py index 62103e24cec4..4a74ebba6df1 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/vertex_ai/hyperparameter_tuning_job.py +++ b/providers/src/airflow/providers/google/cloud/hooks/vertex_ai/hyperparameter_tuning_job.py @@ -52,14 +52,10 @@ def __init__( impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) super().__init__( gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, + **kwargs, ) self._hyperparameter_tuning_job: HyperparameterTuningJob | None = None diff --git a/providers/src/airflow/providers/google/cloud/hooks/vertex_ai/model_service.py b/providers/src/airflow/providers/google/cloud/hooks/vertex_ai/model_service.py index ee3b171eb63a..e1396f217bc2 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/vertex_ai/model_service.py +++ b/providers/src/airflow/providers/google/cloud/hooks/vertex_ai/model_service.py @@ -42,14 +42,6 @@ class ModelServiceHook(GoogleBaseHook): """Hook for Google Cloud Vertex AI Endpoint Service APIs.""" - def __init__(self, **kwargs): - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) - super().__init__(**kwargs) - def get_model_service_client(self, region: str | None = None) -> ModelServiceClient: """Return ModelServiceClient object.""" if region and region != "global": diff --git a/providers/src/airflow/providers/google/cloud/hooks/vertex_ai/pipeline_job.py b/providers/src/airflow/providers/google/cloud/hooks/vertex_ai/pipeline_job.py index f1a5fb40bc86..a1217608ac0d 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/vertex_ai/pipeline_job.py +++ b/providers/src/airflow/providers/google/cloud/hooks/vertex_ai/pipeline_job.py @@ -62,6 +62,7 @@ def __init__( super().__init__( gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, + **kwargs, ) self._pipeline_job: PipelineJob | None = None diff --git a/providers/src/airflow/providers/google/cloud/hooks/video_intelligence.py b/providers/src/airflow/providers/google/cloud/hooks/video_intelligence.py index 8e37ab3e5622..dd0bba6fa863 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/video_intelligence.py +++ b/providers/src/airflow/providers/google/cloud/hooks/video_intelligence.py @@ -60,14 +60,10 @@ def __init__( impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) super().__init__( gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, + **kwargs, ) self._conn: VideoIntelligenceServiceClient | None = None diff --git a/providers/src/airflow/providers/google/cloud/hooks/vision.py b/providers/src/airflow/providers/google/cloud/hooks/vision.py index 47b033e7ef8e..0bf9b4c01573 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/vision.py +++ b/providers/src/airflow/providers/google/cloud/hooks/vision.py @@ -130,14 +130,10 @@ def __init__( impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) super().__init__( gcp_conn_id=gcp_conn_id, impersonation_chain=impersonation_chain, + **kwargs, ) self._client = None diff --git a/providers/src/airflow/providers/google/cloud/hooks/workflows.py b/providers/src/airflow/providers/google/cloud/hooks/workflows.py index a05a9aa36f6f..23cc2dd1ea23 100644 --- a/providers/src/airflow/providers/google/cloud/hooks/workflows.py +++ b/providers/src/airflow/providers/google/cloud/hooks/workflows.py @@ -41,14 +41,6 @@ class WorkflowsHook(GoogleBaseHook): keyword arguments rather than positional. """ - def __init__(self, **kwargs): - if kwargs.get("delegate_to") is not None: - raise RuntimeError( - "The `delegate_to` parameter has been deprecated before and finally removed in this version" - " of Google Provider. You MUST convert it to `impersonate_chain`" - ) - super().__init__(**kwargs) - def get_workflows_client(self) -> WorkflowsClient: """Return WorkflowsClient object.""" return WorkflowsClient(credentials=self.get_credentials(), client_info=CLIENT_INFO) diff --git a/providers/src/airflow/providers/google/cloud/links/dataproc.py b/providers/src/airflow/providers/google/cloud/links/dataproc.py index 5ca4bfdb0a96..556b7c60a4bb 100644 --- a/providers/src/airflow/providers/google/cloud/links/dataproc.py +++ b/providers/src/airflow/providers/google/cloud/links/dataproc.py @@ -121,7 +121,6 @@ def __attrs_post_init__(self): # This link is still used into the selected operators # - airflow.providers.google.cloud.operators.dataproc.DataprocScaleClusterOperator # - airflow.providers.google.cloud.operators.dataproc.DataprocJobBaseOperator - # - airflow.providers.google.cloud.operators.dataproc.DataprocSubmitPigJobOperator # As soon as we remove reference to this link we might deprecate it by add warning message # with `stacklevel=3` below in this method. ... diff --git a/providers/src/airflow/providers/google/cloud/operators/automl.py b/providers/src/airflow/providers/google/cloud/operators/automl.py index a7434d51f43c..89d1c617cffe 100644 --- a/providers/src/airflow/providers/google/cloud/operators/automl.py +++ b/providers/src/airflow/providers/google/cloud/operators/automl.py @@ -85,7 +85,7 @@ class AutoMLTrainModelOperator(GoogleCloudBaseOperator): :class:`airflow.providers.google.cloud.operators.vertex_ai.auto_ml.CreateAutoMLTabularTrainingJobOperator`, :class:`airflow.providers.google.cloud.operators.vertex_ai.auto_ml.CreateAutoMLVideoTrainingJobOperator`, :class:`airflow.providers.google.cloud.operators.vertex_ai.auto_ml.CreateAutoMLImageTrainingJobOperator`, - :class:`airflow.providers.google.cloud.operators.vertex_ai.auto_ml.CreateAutoMLTextTrainingJobOperator`, + :class:`airflow.providers.google.cloud.operators.vertex_ai.generative_model.SupervisedFineTuningTrainOperator`, instead. .. seealso:: @@ -155,7 +155,7 @@ def execute(self, context: Context): "CreateAutoMLTabularTrainingJobOperator", "CreateAutoMLVideoTrainingJobOperator", "CreateAutoMLImageTrainingJobOperator", - "CreateAutoMLTextTrainingJobOperator", + "SupervisedFineTuningTrainOperator", ], ) hook = CloudAutoMLHook( diff --git a/providers/src/airflow/providers/google/cloud/operators/bigquery.py b/providers/src/airflow/providers/google/cloud/operators/bigquery.py index 876ff8d51f16..db5566f769b9 100644 --- a/providers/src/airflow/providers/google/cloud/operators/bigquery.py +++ b/providers/src/airflow/providers/google/cloud/operators/bigquery.py @@ -24,17 +24,14 @@ import re import warnings from functools import cached_property -from typing import TYPE_CHECKING, Any, Iterable, Sequence, SupportsAbs +from typing import TYPE_CHECKING, Any, Sequence, SupportsAbs -import attr from google.api_core.exceptions import Conflict from google.cloud.bigquery import DEFAULT_RETRY, CopyJob, ExtractJob, LoadJob, QueryJob, Row from google.cloud.bigquery.table import RowIterator from airflow.configuration import conf from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning, AirflowSkipException -from airflow.models import BaseOperator, BaseOperatorLink -from airflow.models.xcom import XCom from airflow.providers.common.sql.operators.sql import ( # type: ignore[attr-defined] # for _parse_boolean SQLCheckOperator, SQLColumnCheckOperator, @@ -56,7 +53,6 @@ BigQueryValueCheckTrigger, ) from airflow.providers.google.cloud.utils.bigquery import convert_job_id -from airflow.providers.google.common.deprecated import deprecated from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID from airflow.utils.helpers import exactly_one @@ -64,7 +60,6 @@ from google.api_core.retry import Retry from google.cloud.bigquery import UnknownJob - from airflow.models.taskinstancekey import TaskInstanceKey from airflow.utils.context import Context @@ -91,46 +86,6 @@ class IfExistAction(enum.Enum): SKIP = "skip" -class BigQueryConsoleLink(BaseOperatorLink): - """Helper class for constructing BigQuery link.""" - - name = "BigQuery Console" - - def get_link( - self, - operator: BaseOperator, - *, - ti_key: TaskInstanceKey, - ): - job_id_path = XCom.get_value(key="job_id_path", ti_key=ti_key) - return BIGQUERY_JOB_DETAILS_LINK_FMT.format(job_id=job_id_path) if job_id_path else "" - - -@attr.s(auto_attribs=True) -class BigQueryConsoleIndexableLink(BaseOperatorLink): - """Helper class for constructing BigQuery link.""" - - index: int = attr.ib() - - @property - def name(self) -> str: - return f"BigQuery Console #{self.index + 1}" - - def get_link( - self, - operator: BaseOperator, - *, - ti_key: TaskInstanceKey, - ): - job_ids = XCom.get_value(key="job_id_path", ti_key=ti_key) - if not job_ids: - return None - if len(job_ids) < self.index: - return None - job_id = job_ids[self.index] - return BIGQUERY_JOB_DETAILS_LINK_FMT.format(job_id=job_id) - - class _BigQueryDbHookMixin: def get_db_hook(self: BigQueryCheckOperator) -> BigQueryHook: # type:ignore[misc] """Get BigQuery DB Hook.""" @@ -1202,231 +1157,6 @@ def execute_complete(self, context: Context, event: dict[str, Any]) -> Any: return event["records"] -@deprecated( - planned_removal_date="November 01, 2024", - use_instead="BigQueryInsertJobOperator", - category=AirflowProviderDeprecationWarning, -) -class BigQueryExecuteQueryOperator(GoogleCloudBaseOperator): - """ - Executes BigQuery SQL queries in a specific BigQuery database. - - This operator is deprecated. Please use - :class:`airflow.providers.google.cloud.operators.bigquery.BigQueryInsertJobOperator` - instead. - - This operator does not assert idempotency. - - :param sql: the SQL code to be executed as a single string, or - a list of str (sql statements), or a reference to a template file. - Template references are recognized by str ending in '.sql' - :param destination_dataset_table: A dotted - ``(.|:).
`` that, if set, will store the results - of the query. (templated) - :param write_disposition: Specifies the action that occurs if the destination table - already exists. (default: 'WRITE_EMPTY') - :param create_disposition: Specifies whether the job is allowed to create new tables. - (default: 'CREATE_IF_NEEDED') - :param allow_large_results: Whether to allow large results. - :param flatten_results: If true and query uses legacy SQL dialect, flattens - all nested and repeated fields in the query results. ``allow_large_results`` - must be ``true`` if this is set to ``false``. For standard SQL queries, this - flag is ignored and results are never flattened. - :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud. - :param udf_config: The User Defined Function configuration for the query. - See https://cloud.google.com/bigquery/user-defined-functions for details. - :param use_legacy_sql: Whether to use legacy SQL (true) or standard SQL (false). - :param maximum_billing_tier: Positive integer that serves as a multiplier - of the basic price. - Defaults to None, in which case it uses the value set in the project. - :param maximum_bytes_billed: Limits the bytes billed for this job. - Queries that will have bytes billed beyond this limit will fail - (without incurring a charge). If unspecified, this will be - set to your project default. - :param api_resource_configs: a dictionary that contain params - 'configuration' applied for Google BigQuery Jobs API: - https://cloud.google.com/bigquery/docs/reference/rest/v2/jobs - for example, {'query': {'useQueryCache': False}}. You could use it - if you need to provide some params that are not supported by BigQueryOperator - like args. - :param schema_update_options: Allows the schema of the destination - table to be updated as a side effect of the load job. - :param query_params: a list of dictionary containing query parameter types and - values, passed to BigQuery. The structure of dictionary should look like - 'queryParameters' in Google BigQuery Jobs API: - https://cloud.google.com/bigquery/docs/reference/rest/v2/jobs. - For example, [{ 'name': 'corpus', 'parameterType': { 'type': 'STRING' }, - 'parameterValue': { 'value': 'romeoandjuliet' } }]. (templated) - :param labels: a dictionary containing labels for the job/query, - passed to BigQuery - :param priority: Specifies a priority for the query. - Possible values include INTERACTIVE and BATCH. - The default value is INTERACTIVE. - :param time_partitioning: configure optional time partitioning fields i.e. - partition by field, type and expiration as per API specifications. - :param cluster_fields: Request that the result of this query be stored sorted - by one or more columns. BigQuery supports clustering for both partitioned and - non-partitioned tables. The order of columns given determines the sort order. - :param location: The geographic location of the job. Required except for - US and EU. See details at - https://cloud.google.com/bigquery/docs/locations#specifying_your_location - :param encryption_configuration: (Optional) Custom encryption configuration (e.g., Cloud KMS keys). - - .. code-block:: python - - encryption_configuration = { - "kmsKeyName": "projects/PROJECT/locations/LOCATION/keyRings/KEY_RING/cryptoKeys/KEY", - } - :param impersonation_chain: Optional service account to impersonate using short-term - credentials, or chained list of accounts required to get the access_token - of the last account in the list, which will be impersonated in the request. - If set as a string, the account must grant the originating account - the Service Account Token Creator IAM role. - If set as a sequence, the identities from the list must grant - Service Account Token Creator IAM role to the directly preceding identity, with first - account from the list granting this role to the originating account (templated). - """ - - template_fields: Sequence[str] = ( - "sql", - "destination_dataset_table", - "labels", - "query_params", - "impersonation_chain", - ) - template_ext: Sequence[str] = (".sql",) - template_fields_renderers = {"sql": "sql"} - ui_color = BigQueryUIColors.QUERY.value - - @property - def operator_extra_links(self): - """Return operator extra links.""" - if isinstance(self.sql, str): - return (BigQueryConsoleLink(),) - return (BigQueryConsoleIndexableLink(i) for i, _ in enumerate(self.sql)) - - def __init__( - self, - *, - sql: str | Iterable[str], - destination_dataset_table: str | None = None, - write_disposition: str = "WRITE_EMPTY", - allow_large_results: bool = False, - flatten_results: bool | None = None, - gcp_conn_id: str = "google_cloud_default", - udf_config: list | None = None, - use_legacy_sql: bool = True, - maximum_billing_tier: int | None = None, - maximum_bytes_billed: float | None = None, - create_disposition: str = "CREATE_IF_NEEDED", - schema_update_options: list | tuple | set | None = None, - query_params: list | None = None, - labels: dict | None = None, - priority: str = "INTERACTIVE", - time_partitioning: dict | None = None, - api_resource_configs: dict | None = None, - cluster_fields: list[str] | None = None, - location: str | None = None, - encryption_configuration: dict | None = None, - impersonation_chain: str | Sequence[str] | None = None, - impersonation_scopes: str | Sequence[str] | None = None, - job_id: str | list[str] | None = None, - **kwargs, - ) -> None: - super().__init__(**kwargs) - self.sql = sql - self.destination_dataset_table = destination_dataset_table - self.write_disposition = write_disposition - self.create_disposition = create_disposition - self.allow_large_results = allow_large_results - self.flatten_results = flatten_results - self.gcp_conn_id = gcp_conn_id - self.udf_config = udf_config - self.use_legacy_sql = use_legacy_sql - self.maximum_billing_tier = maximum_billing_tier - self.maximum_bytes_billed = maximum_bytes_billed - self.schema_update_options = schema_update_options - self.query_params = query_params - self.labels = labels - self.priority = priority - self.time_partitioning = time_partitioning - self.api_resource_configs = api_resource_configs - self.cluster_fields = cluster_fields - self.location = location - self.encryption_configuration = encryption_configuration - self.hook: BigQueryHook | None = None - self.impersonation_chain = impersonation_chain - self.impersonation_scopes = impersonation_scopes - self.job_id = job_id - - def execute(self, context: Context): - if self.hook is None: - self.log.info("Executing: %s", self.sql) - self.hook = BigQueryHook( - gcp_conn_id=self.gcp_conn_id, - use_legacy_sql=self.use_legacy_sql, - location=self.location, - impersonation_chain=self.impersonation_chain, - impersonation_scopes=self.impersonation_scopes, - ) - if isinstance(self.sql, str): - self.job_id = self.hook.run_query( - sql=self.sql, - destination_dataset_table=self.destination_dataset_table, - write_disposition=self.write_disposition, - allow_large_results=self.allow_large_results, - flatten_results=self.flatten_results, - udf_config=self.udf_config, - maximum_billing_tier=self.maximum_billing_tier, - maximum_bytes_billed=self.maximum_bytes_billed, - create_disposition=self.create_disposition, - query_params=self.query_params, - labels=self.labels, - schema_update_options=self.schema_update_options, - priority=self.priority, - time_partitioning=self.time_partitioning, - api_resource_configs=self.api_resource_configs, - cluster_fields=self.cluster_fields, - encryption_configuration=self.encryption_configuration, - ) - elif isinstance(self.sql, Iterable): - self.job_id = [ - self.hook.run_query( - sql=s, - destination_dataset_table=self.destination_dataset_table, - write_disposition=self.write_disposition, - allow_large_results=self.allow_large_results, - flatten_results=self.flatten_results, - udf_config=self.udf_config, - maximum_billing_tier=self.maximum_billing_tier, - maximum_bytes_billed=self.maximum_bytes_billed, - create_disposition=self.create_disposition, - query_params=self.query_params, - labels=self.labels, - schema_update_options=self.schema_update_options, - priority=self.priority, - time_partitioning=self.time_partitioning, - api_resource_configs=self.api_resource_configs, - cluster_fields=self.cluster_fields, - encryption_configuration=self.encryption_configuration, - ) - for s in self.sql - ] - else: - raise AirflowException(f"argument 'sql' of type {type(str)} is neither a string nor an iterable") - project_id = self.hook.project_id - if project_id: - job_id_path = convert_job_id(job_id=self.job_id, project_id=project_id, location=self.location) # type: ignore[arg-type] - context["task_instance"].xcom_push(key="job_id_path", value=job_id_path) - return self.job_id - - def on_kill(self) -> None: - super().on_kill() - if self.hook is not None: - self.log.info("Cancelling running query") - self.hook.cancel_job(self.hook.running_job_id) - - class BigQueryCreateEmptyTableOperator(GoogleCloudBaseOperator): """ Creates a new table in the specified BigQuery dataset, optionally with schema. @@ -2298,76 +2028,6 @@ def execute(self, context: Context): ) -@deprecated( - planned_removal_date="November 01, 2024", - use_instead="BigQueryUpdateDatasetOperator", - category=AirflowProviderDeprecationWarning, -) -class BigQueryPatchDatasetOperator(GoogleCloudBaseOperator): - """ - Patch a dataset for your Project in BigQuery. - - This operator is deprecated. Please use - :class:`airflow.providers.google.cloud.operators.bigquery.BigQueryUpdateTableOperator` - instead. - - Only replaces fields that are provided in the submitted dataset resource. - - :param dataset_id: The id of dataset. Don't need to provide, - if datasetId in dataset_reference. - :param dataset_resource: Dataset resource that will be provided with request body. - https://cloud.google.com/bigquery/docs/reference/rest/v2/datasets#resource - :param project_id: The name of the project where we want to create the dataset. - Don't need to provide, if projectId in dataset_reference. - :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud. - :param impersonation_chain: Optional service account to impersonate using short-term - credentials, or chained list of accounts required to get the access_token - of the last account in the list, which will be impersonated in the request. - If set as a string, the account must grant the originating account - the Service Account Token Creator IAM role. - If set as a sequence, the identities from the list must grant - Service Account Token Creator IAM role to the directly preceding identity, with first - account from the list granting this role to the originating account (templated). - """ - - template_fields: Sequence[str] = ( - "dataset_id", - "project_id", - "impersonation_chain", - ) - template_fields_renderers = {"dataset_resource": "json"} - ui_color = BigQueryUIColors.DATASET.value - - def __init__( - self, - *, - dataset_id: str, - dataset_resource: dict, - project_id: str = PROVIDE_PROJECT_ID, - gcp_conn_id: str = "google_cloud_default", - impersonation_chain: str | Sequence[str] | None = None, - **kwargs, - ) -> None: - self.dataset_id = dataset_id - self.project_id = project_id - self.gcp_conn_id = gcp_conn_id - self.dataset_resource = dataset_resource - self.impersonation_chain = impersonation_chain - super().__init__(**kwargs) - - def execute(self, context: Context): - bq_hook = BigQueryHook( - gcp_conn_id=self.gcp_conn_id, - impersonation_chain=self.impersonation_chain, - ) - - return bq_hook.patch_dataset( - dataset_id=self.dataset_id, - dataset_resource=self.dataset_resource, - project_id=self.project_id, - ) - - class BigQueryUpdateTableOperator(GoogleCloudBaseOperator): """ Update a table for your Project in BigQuery. diff --git a/providers/src/airflow/providers/google/cloud/operators/dataflow.py b/providers/src/airflow/providers/google/cloud/operators/dataflow.py index 29ac5943edfe..34a25042ad7c 100644 --- a/providers/src/airflow/providers/google/cloud/operators/dataflow.py +++ b/providers/src/airflow/providers/google/cloud/operators/dataflow.py @@ -19,10 +19,7 @@ from __future__ import annotations -import copy -import re import uuid -from contextlib import ExitStack from enum import Enum from functools import cached_property from typing import TYPE_CHECKING, Any, Sequence @@ -31,13 +28,10 @@ from airflow.configuration import conf from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning -from airflow.providers.apache.beam.hooks.beam import BeamHook, BeamRunnerType from airflow.providers.google.cloud.hooks.dataflow import ( DEFAULT_DATAFLOW_LOCATION, DataflowHook, - process_line_and_extract_dataflow_job_id_callback, ) -from airflow.providers.google.cloud.hooks.gcs import GCSHook from airflow.providers.google.cloud.links.dataflow import DataflowJobLink, DataflowPipelineLink from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator from airflow.providers.google.cloud.triggers.dataflow import ( @@ -47,7 +41,6 @@ from airflow.providers.google.common.consts import GOOGLE_DEFAULT_DEFERRABLE_METHOD_NAME from airflow.providers.google.common.deprecated import deprecated from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID -from airflow.version import version if TYPE_CHECKING: from airflow.utils.context import Context @@ -175,297 +168,6 @@ def __init__( self.service_account = service_account -# TODO: Remove one day -@deprecated( - planned_removal_date="November 01, 2024", - use_instead="providers.apache.beam.operators.beam.BeamRunJavaPipelineOperator", - category=AirflowProviderDeprecationWarning, -) -class DataflowCreateJavaJobOperator(GoogleCloudBaseOperator): - """ - Start a Java Cloud Dataflow batch job; the parameters of the operation will be passed to the job. - - This class is deprecated. - - Please use :class:`providers.apache.beam.operators.beam.BeamRunJavaPipelineOperator`. - - Example usage: - - .. code-block:: python - - default_args = { - "owner": "airflow", - "depends_on_past": False, - "start_date": (2016, 8, 1), - "email": ["alex@vanboxel.be"], - "email_on_failure": False, - "email_on_retry": False, - "retries": 1, - "retry_delay": timedelta(minutes=30), - "dataflow_default_options": { - "project": "my-gcp-project", - "zone": "us-central1-f", - "stagingLocation": "gs://bucket/tmp/dataflow/staging/", - }, - } - - dag = DAG("test-dag", default_args=default_args) - - task = DataflowCreateJavaJobOperator( - gcp_conn_id="gcp_default", - task_id="normalize-cal", - jar="{{var.value.gcp_dataflow_base}}pipeline-ingress-cal-normalize-1.0.jar", - options={ - "autoscalingAlgorithm": "BASIC", - "maxNumWorkers": "50", - "start": "{{ds}}", - "partitionType": "DAY", - }, - dag=dag, - ) - - - .. seealso:: - For more detail on job submission have a look at the reference: - https://cloud.google.com/dataflow/pipelines/specifying-exec-params - - .. seealso:: - For more information on how to use this operator, take a look at the guide: - :ref:`howto/operator:DataflowCreateJavaJobOperator` - - :param jar: The reference to a self executing Dataflow jar (templated). - :param job_name: The 'jobName' to use when executing the Dataflow job - (templated). This ends up being set in the pipeline options, so any entry - with key ``'jobName'`` in ``options`` will be overwritten. - :param dataflow_default_options: Map of default job options. - :param options: Map of job specific options.The key must be a dictionary. - - The value can contain different types: - - * If the value is None, the single option - ``--key`` (without value) will be added. - * If the value is False, this option will be skipped - * If the value is True, the single option - ``--key`` (without value) will be added. - * If the value is list, the many options will be added for each key. - If the value is ``['A', 'B']`` and the key is ``key`` then the ``--key=A --key=B`` options - will be left - * Other value types will be replaced with the Python textual representation. - - When defining labels (``labels`` option), you can also provide a dictionary. - - :param project_id: Optional, the Google Cloud project ID in which to start a job. - If set to None or missing, the default project_id from the Google Cloud connection is used. - :param location: Job location. - :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param poll_sleep: The time in seconds to sleep between polling Google - Cloud Platform for the dataflow job status while the job is in the - JOB_STATE_RUNNING state. - :param job_class: The name of the dataflow job class to be executed, it - is often not the main class configured in the dataflow jar file. - - :param multiple_jobs: If pipeline creates multiple jobs then monitor all jobs - :param check_if_running: before running job, validate that a previous run is not in process - if job is running finish with nothing, WaitForRun= wait until job finished and the run job) - ``jar``, ``options``, and ``job_name`` are templated so you can use variables in them. - :param cancel_timeout: How long (in seconds) operator should wait for the pipeline to be - successfully cancelled when task is being killed. - :param wait_until_finished: (Optional) - If True, wait for the end of pipeline execution before exiting. - If False, only submits job. - If None, default behavior. - - The default behavior depends on the type of pipeline: - - * for the streaming pipeline, wait for jobs to start, - * for the batch pipeline, wait for the jobs to complete. - - .. warning:: - - You cannot call ``PipelineResult.wait_until_finish`` method in your pipeline code for the operator - to work properly. i. e. you must use asynchronous execution. Otherwise, your pipeline will - always wait until finished. For more information, look at: - `Asynchronous execution - `__ - - The process of starting the Dataflow job in Airflow consists of two steps: - - * running a subprocess and reading the stderr/stderr log for the job id. - * loop waiting for the end of the job ID from the previous step. - This loop checks the status of the job. - - Step two is started just after step one has finished, so if you have wait_until_finished in your - pipeline code, step two will not start until the process stops. When this process stops, - steps two will run, but it will only execute one iteration as the job will be in a terminal state. - - If you in your pipeline do not call the wait_for_pipeline method but pass wait_until_finish=True - to the operator, the second loop will wait for the job's terminal state. - - If you in your pipeline do not call the wait_for_pipeline method, and pass wait_until_finish=False - to the operator, the second loop will check once is job not in terminal state and exit the loop. - :param expected_terminal_state: The expected terminal state of the operator on which the corresponding - Airflow task succeeds. When not specified, it will be determined by the hook. - - Note that both - ``dataflow_default_options`` and ``options`` will be merged to specify pipeline - execution parameter, and ``dataflow_default_options`` is expected to save - high-level options, for instances, project and zone information, which - apply to all dataflow operators in the DAG. - - It's a good practice to define dataflow_* parameters in the default_args of the dag - like the project, zone and staging location. - - .. code-block:: python - - default_args = { - "dataflow_default_options": { - "zone": "europe-west1-d", - "stagingLocation": "gs://my-staging-bucket/staging/", - } - } - - You need to pass the path to your dataflow as a file reference with the ``jar`` - parameter, the jar needs to be a self executing jar (see documentation here: - https://beam.apache.org/documentation/runners/dataflow/#self-executing-jar). - Use ``options`` to pass on options to your job. - - .. code-block:: python - - t1 = DataflowCreateJavaJobOperator( - task_id="dataflow_example", - jar="{{var.value.gcp_dataflow_base}}pipeline/build/libs/pipeline-example-1.0.jar", - options={ - "autoscalingAlgorithm": "BASIC", - "maxNumWorkers": "50", - "start": "{{ds}}", - "partitionType": "DAY", - "labels": {"foo": "bar"}, - }, - gcp_conn_id="airflow-conn-id", - dag=my_dag, - ) - - """ - - template_fields: Sequence[str] = ("options", "jar", "job_name") - ui_color = "#0273d4" - - def __init__( - self, - *, - jar: str, - job_name: str = "{{task.task_id}}", - dataflow_default_options: dict | None = None, - options: dict | None = None, - project_id: str = PROVIDE_PROJECT_ID, - location: str = DEFAULT_DATAFLOW_LOCATION, - gcp_conn_id: str = "google_cloud_default", - poll_sleep: int = 10, - job_class: str | None = None, - check_if_running: CheckJobRunning = CheckJobRunning.WaitForRun, - multiple_jobs: bool = False, - cancel_timeout: int | None = 10 * 60, - wait_until_finished: bool | None = None, - expected_terminal_state: str | None = None, - **kwargs, - ) -> None: - super().__init__(**kwargs) - - dataflow_default_options = dataflow_default_options or {} - options = options or {} - options.setdefault("labels", {}).update( - {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")} - ) - self.project_id = project_id - self.location = location - self.gcp_conn_id = gcp_conn_id - self.jar = jar - self.multiple_jobs = multiple_jobs - self.job_name = job_name - self.dataflow_default_options = dataflow_default_options - self.options = options - self.poll_sleep = poll_sleep - self.job_class = job_class - self.check_if_running = check_if_running - self.cancel_timeout = cancel_timeout - self.wait_until_finished = wait_until_finished - self.expected_terminal_state = expected_terminal_state - self.job_id = None - self.beam_hook: BeamHook | None = None - self.dataflow_hook: DataflowHook | None = None - - def execute(self, context: Context): - """Execute the Apache Beam Pipeline.""" - self.beam_hook = BeamHook(runner=BeamRunnerType.DataflowRunner) - self.dataflow_hook = DataflowHook( - gcp_conn_id=self.gcp_conn_id, - poll_sleep=self.poll_sleep, - cancel_timeout=self.cancel_timeout, - wait_until_finished=self.wait_until_finished, - expected_terminal_state=self.expected_terminal_state, - ) - job_name = self.dataflow_hook.build_dataflow_job_name(job_name=self.job_name) - pipeline_options = copy.deepcopy(self.dataflow_default_options) - - pipeline_options["jobName"] = self.job_name - pipeline_options["project"] = self.project_id or self.dataflow_hook.project_id - pipeline_options["region"] = self.location - pipeline_options.update(self.options) - pipeline_options.setdefault("labels", {}).update( - {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")} - ) - pipeline_options.update(self.options) - - def set_current_job_id(job_id): - self.job_id = job_id - - process_line_callback = process_line_and_extract_dataflow_job_id_callback( - on_new_job_id_callback=set_current_job_id - ) - - with ExitStack() as exit_stack: - if self.jar.lower().startswith("gs://"): - gcs_hook = GCSHook(self.gcp_conn_id) - tmp_gcs_file = exit_stack.enter_context(gcs_hook.provide_file(object_url=self.jar)) - self.jar = tmp_gcs_file.name - - is_running = False - if self.check_if_running != CheckJobRunning.IgnoreJob: - is_running = self.dataflow_hook.is_job_dataflow_running( - name=self.job_name, - variables=pipeline_options, - location=self.location, - ) - while is_running and self.check_if_running == CheckJobRunning.WaitForRun: - is_running = self.dataflow_hook.is_job_dataflow_running( - name=self.job_name, - variables=pipeline_options, - location=self.location, - ) - if not is_running: - pipeline_options["jobName"] = job_name - with self.dataflow_hook.provide_authorized_gcloud(): - self.beam_hook.start_java_pipeline( - variables=pipeline_options, - jar=self.jar, - job_class=self.job_class, - process_line_callback=process_line_callback, - ) - self.dataflow_hook.wait_for_done( - job_name=job_name, - location=self.location, - job_id=self.job_id, - multiple_jobs=self.multiple_jobs, - ) - - return {"job_id": self.job_id} - - def on_kill(self) -> None: - self.log.info("On kill.") - if self.job_id: - self.dataflow_hook.cancel_job( - job_id=self.job_id, project_id=self.project_id or self.dataflow_hook.project_id - ) - - class DataflowTemplatedJobStartOperator(GoogleCloudBaseOperator): """ Start a Dataflow job with a classic template; the parameters of the operation will be passed to the job. @@ -1233,224 +935,6 @@ def hook(self) -> DataflowHook: ) -# TODO: Remove one day -@deprecated( - planned_removal_date="November 01, 2024", - use_instead="providers.apache.beam.operators.beam.BeamRunPythonPipelineOperator", - category=AirflowProviderDeprecationWarning, -) -class DataflowCreatePythonJobOperator(GoogleCloudBaseOperator): - """ - Launching Cloud Dataflow jobs written in python. - - Note that both dataflow_default_options and options will be merged to specify pipeline - execution parameter, and dataflow_default_options is expected to save high-level options, - for instances, project and zone information, which apply to all dataflow operators in the DAG. - - This class is deprecated. - - Please use :class:`providers.apache.beam.operators.beam.BeamRunPythonPipelineOperator`. - - .. seealso:: - For more detail on job submission have a look at the reference: - https://cloud.google.com/dataflow/pipelines/specifying-exec-params - - .. seealso:: - For more information on how to use this operator, take a look at the guide: - :ref:`howto/operator:DataflowCreatePythonJobOperator` - - :param py_file: Reference to the python dataflow pipeline file.py, e.g., - /some/local/file/path/to/your/python/pipeline/file. (templated) - :param job_name: The 'job_name' to use when executing the Dataflow job - (templated). This ends up being set in the pipeline options, so any entry - with key ``'jobName'`` or ``'job_name'`` in ``options`` will be overwritten. - :param py_options: Additional python options, e.g., ["-m", "-v"]. - :param dataflow_default_options: Map of default job options. - :param options: Map of job specific options.The key must be a dictionary. - The value can contain different types: - - * If the value is None, the single option - ``--key`` (without value) will be added. - * If the value is False, this option will be skipped - * If the value is True, the single option - ``--key`` (without value) will be added. - * If the value is list, the many options will be added for each key. - If the value is ``['A', 'B']`` and the key is ``key`` then the ``--key=A --key=B`` options - will be left - * Other value types will be replaced with the Python textual representation. - - When defining labels (``labels`` option), you can also provide a dictionary. - :param py_interpreter: Python version of the beam pipeline. - If None, this defaults to the python3. - To track python versions supported by beam and related - issues check: https://issues.apache.org/jira/browse/BEAM-1251 - :param py_requirements: Additional python package(s) to install. - If a value is passed to this parameter, a new virtual environment has been created with - additional packages installed. - - You could also install the apache_beam package if it is not installed on your system or you want - to use a different version. - :param py_system_site_packages: Whether to include system_site_packages in your virtualenv. - See virtualenv documentation for more information. - - This option is only relevant if the ``py_requirements`` parameter is not None. - :param gcp_conn_id: The connection ID to use connecting to Google Cloud. - :param project_id: Optional, the Google Cloud project ID in which to start a job. - If set to None or missing, the default project_id from the Google Cloud connection is used. - :param location: Job location. - :param poll_sleep: The time in seconds to sleep between polling Google - Cloud Platform for the dataflow job status while the job is in the - JOB_STATE_RUNNING state. - :param drain_pipeline: Optional, set to True if want to stop streaming job by draining it - instead of canceling during killing task instance. See: - https://cloud.google.com/dataflow/docs/guides/stopping-a-pipeline - :param cancel_timeout: How long (in seconds) operator should wait for the pipeline to be - successfully cancelled when task is being killed. - :param wait_until_finished: (Optional) - If True, wait for the end of pipeline execution before exiting. - If False, only submits job. - If None, default behavior. - - The default behavior depends on the type of pipeline: - - * for the streaming pipeline, wait for jobs to start, - * for the batch pipeline, wait for the jobs to complete. - - .. warning:: - - You cannot call ``PipelineResult.wait_until_finish`` method in your pipeline code for the operator - to work properly. i. e. you must use asynchronous execution. Otherwise, your pipeline will - always wait until finished. For more information, look at: - `Asynchronous execution - `__ - - The process of starting the Dataflow job in Airflow consists of two steps: - - * running a subprocess and reading the stderr/stderr log for the job id. - * loop waiting for the end of the job ID from the previous step. - This loop checks the status of the job. - - Step two is started just after step one has finished, so if you have wait_until_finished in your - pipeline code, step two will not start until the process stops. When this process stops, - steps two will run, but it will only execute one iteration as the job will be in a terminal state. - - If you in your pipeline do not call the wait_for_pipeline method but pass wait_until_finish=True - to the operator, the second loop will wait for the job's terminal state. - - If you in your pipeline do not call the wait_for_pipeline method, and pass wait_until_finish=False - to the operator, the second loop will check once is job not in terminal state and exit the loop. - """ - - template_fields: Sequence[str] = ("options", "dataflow_default_options", "job_name", "py_file") - - def __init__( - self, - *, - py_file: str, - job_name: str = "{{task.task_id}}", - dataflow_default_options: dict | None = None, - options: dict | None = None, - py_interpreter: str = "python3", - py_options: list[str] | None = None, - py_requirements: list[str] | None = None, - py_system_site_packages: bool = False, - project_id: str = PROVIDE_PROJECT_ID, - location: str = DEFAULT_DATAFLOW_LOCATION, - gcp_conn_id: str = "google_cloud_default", - poll_sleep: int = 10, - drain_pipeline: bool = False, - cancel_timeout: int | None = 10 * 60, - wait_until_finished: bool | None = None, - **kwargs, - ) -> None: - super().__init__(**kwargs) - - self.py_file = py_file - self.job_name = job_name - self.py_options = py_options or [] - self.dataflow_default_options = dataflow_default_options or {} - self.options = options or {} - self.options.setdefault("labels", {}).update( - {"airflow-version": "v" + version.replace(".", "-").replace("+", "-")} - ) - self.py_interpreter = py_interpreter - self.py_requirements = py_requirements - self.py_system_site_packages = py_system_site_packages - self.project_id = project_id - self.location = location - self.gcp_conn_id = gcp_conn_id - self.poll_sleep = poll_sleep - self.drain_pipeline = drain_pipeline - self.cancel_timeout = cancel_timeout - self.wait_until_finished = wait_until_finished - self.job_id = None - self.beam_hook: BeamHook | None = None - self.dataflow_hook: DataflowHook | None = None - - def execute(self, context: Context): - """Execute the python dataflow job.""" - self.beam_hook = BeamHook(runner=BeamRunnerType.DataflowRunner) - self.dataflow_hook = DataflowHook( - gcp_conn_id=self.gcp_conn_id, - poll_sleep=self.poll_sleep, - impersonation_chain=None, - drain_pipeline=self.drain_pipeline, - cancel_timeout=self.cancel_timeout, - wait_until_finished=self.wait_until_finished, - ) - - job_name = self.dataflow_hook.build_dataflow_job_name(job_name=self.job_name) - pipeline_options = self.dataflow_default_options.copy() - pipeline_options["job_name"] = job_name - pipeline_options["project"] = self.project_id or self.dataflow_hook.project_id - pipeline_options["region"] = self.location - pipeline_options.update(self.options) - - # Convert argument names from lowerCamelCase to snake case. - def camel_to_snake(name): - return re.sub("[A-Z]", lambda x: "_" + x.group(0).lower(), name) - - formatted_pipeline_options = {camel_to_snake(key): pipeline_options[key] for key in pipeline_options} - - def set_current_job_id(job_id): - self.job_id = job_id - - process_line_callback = process_line_and_extract_dataflow_job_id_callback( - on_new_job_id_callback=set_current_job_id - ) - - with ExitStack() as exit_stack: - if self.py_file.lower().startswith("gs://"): - gcs_hook = GCSHook(self.gcp_conn_id) - tmp_gcs_file = exit_stack.enter_context(gcs_hook.provide_file(object_url=self.py_file)) - self.py_file = tmp_gcs_file.name - - with self.dataflow_hook.provide_authorized_gcloud(): - self.beam_hook.start_python_pipeline( - variables=formatted_pipeline_options, - py_file=self.py_file, - py_options=self.py_options, - py_interpreter=self.py_interpreter, - py_requirements=self.py_requirements, - py_system_site_packages=self.py_system_site_packages, - process_line_callback=process_line_callback, - ) - - self.dataflow_hook.wait_for_done( - job_name=job_name, - location=self.location, - job_id=self.job_id, - multiple_jobs=False, - ) - - return {"job_id": self.job_id} - - def on_kill(self) -> None: - self.log.info("On kill.") - if self.job_id: - self.dataflow_hook.cancel_job( - job_id=self.job_id, project_id=self.project_id or self.dataflow_hook.project_id - ) - - class DataflowStopJobOperator(GoogleCloudBaseOperator): """ Stops the job with the specified name prefix or Job ID. diff --git a/providers/src/airflow/providers/google/cloud/operators/dataproc.py b/providers/src/airflow/providers/google/cloud/operators/dataproc.py index 13be5b240cfc..270114e5e53a 100644 --- a/providers/src/airflow/providers/google/cloud/operators/dataproc.py +++ b/providers/src/airflow/providers/google/cloud/operators/dataproc.py @@ -20,11 +20,8 @@ from __future__ import annotations import inspect -import ntpath -import os import re import time -import uuid import warnings from collections.abc import MutableSequence from dataclasses import dataclass @@ -45,7 +42,6 @@ DataProcJobBuilder, DataprocResourceIsNotReadyError, ) -from airflow.providers.google.cloud.hooks.gcs import GCSHook from airflow.providers.google.cloud.links.dataproc import ( DATAPROC_BATCH_LINK, DATAPROC_CLUSTER_LINK_DEPRECATED, @@ -1547,632 +1543,6 @@ def on_kill(self) -> None: self.hook.cancel_job(project_id=self.project_id, job_id=self.dataproc_job_id, region=self.region) -@deprecated( - planned_removal_date="November 01, 2024", - use_instead="DataprocSubmitJobOperator", - instructions="You can use `generate_job` method to generate dictionary representing your job " - "and use it with the new operator.", - category=AirflowProviderDeprecationWarning, -) -class DataprocSubmitPigJobOperator(DataprocJobBaseOperator): - """ - Start a Pig query Job on a Cloud DataProc cluster. - - .. seealso:: - This operator is deprecated, please use - :class:`~airflow.providers.google.cloud.operators.dataproc.DataprocSubmitJobOperator`: - - The parameters of the operation will be passed to the cluster. - - It's a good practice to define dataproc_* parameters in the default_args of the dag - like the cluster name and UDFs. - - .. code-block:: python - - default_args = { - "cluster_name": "cluster-1", - "dataproc_pig_jars": [ - "gs://example/udf/jar/datafu/1.2.0/datafu.jar", - "gs://example/udf/jar/gpig/1.2/gpig.jar", - ], - } - - You can pass a pig script as string or file reference. Use variables to pass on - variables for the pig script to be resolved on the cluster or use the parameters to - be resolved in the script as template parameters. - - .. code-block:: python - - t1 = DataProcPigOperator( - task_id="dataproc_pig", - query="a_pig_script.pig", - variables={"out": "gs://example/output/{{ds}}"}, - ) - - .. seealso:: - For more detail on about job submission have a look at the reference: - https://cloud.google.com/dataproc/reference/rest/v1/projects.regions.jobs - - :param query: The query or reference to the query - file (pg or pig extension). (templated) - :param query_uri: The HCFS URI of the script that contains the Pig queries. - :param variables: Map of named parameters for the query. (templated) - """ - - template_fields: Sequence[str] = ( - "query", - "variables", - "job_name", - "cluster_name", - "region", - "dataproc_jars", - "dataproc_properties", - "impersonation_chain", - ) - template_ext = (".pg", ".pig") - ui_color = "#0273d4" - job_type = "pig_job" - - operator_extra_links = (DataprocLink(),) - - def __init__( - self, - *, - query: str | None = None, - query_uri: str | None = None, - variables: dict | None = None, - impersonation_chain: str | Sequence[str] | None = None, - region: str, - job_name: str = "{{task.task_id}}_{{ds_nodash}}", - cluster_name: str = "cluster-1", - dataproc_properties: dict | None = None, - dataproc_jars: list[str] | None = None, - **kwargs, - ) -> None: - super().__init__( - impersonation_chain=impersonation_chain, - region=region, - job_name=job_name, - cluster_name=cluster_name, - dataproc_properties=dataproc_properties, - dataproc_jars=dataproc_jars, - **kwargs, - ) - self.query = query - self.query_uri = query_uri - self.variables = variables - - def generate_job(self): - """ - Act as a helper method for easier migration to `DataprocSubmitJobOperator`. - - :return: Dict representing Dataproc job - """ - job_template = self.create_job_template() - - if self.query is None: - if self.query_uri is None: - raise AirflowException("One of query or query_uri should be set here") - job_template.add_query_uri(self.query_uri) - else: - job_template.add_query(self.query) - job_template.add_variables(self.variables) - return self._generate_job_template() - - def execute(self, context: Context): - job_template = self.create_job_template() - if self.query is None: - if self.query_uri is None: - raise AirflowException("One of query or query_uri should be set here") - job_template.add_query_uri(self.query_uri) - else: - job_template.add_query(self.query) - job_template.add_variables(self.variables) - - super().execute(context) - - -# TODO: Remove one day - - -# TODO: Remove one day -@deprecated( - planned_removal_date="November 01, 2024", - use_instead="DataprocSubmitJobOperator", - instructions="You can use `generate_job` method to generate dictionary representing your job " - "and use it with the new operator.", - category=AirflowProviderDeprecationWarning, -) -class DataprocSubmitHiveJobOperator(DataprocJobBaseOperator): - """ - Start a Hive query Job on a Cloud DataProc cluster. - - .. seealso:: - This operator is deprecated, please use - :class:`~airflow.providers.google.cloud.operators.dataproc.DataprocSubmitJobOperator`: - - :param query: The query or reference to the query file (q extension). - :param query_uri: The HCFS URI of the script that contains the Hive queries. - :param variables: Map of named parameters for the query. - """ - - template_fields: Sequence[str] = ( - "query", - "variables", - "job_name", - "cluster_name", - "region", - "dataproc_jars", - "dataproc_properties", - "impersonation_chain", - ) - template_ext = (".q", ".hql") - ui_color = "#0273d4" - job_type = "hive_job" - - def __init__( - self, - *, - query: str | None = None, - query_uri: str | None = None, - variables: dict | None = None, - impersonation_chain: str | Sequence[str] | None = None, - region: str, - job_name: str = "{{task.task_id}}_{{ds_nodash}}", - cluster_name: str = "cluster-1", - dataproc_properties: dict | None = None, - dataproc_jars: list[str] | None = None, - **kwargs, - ) -> None: - super().__init__( - impersonation_chain=impersonation_chain, - region=region, - job_name=job_name, - cluster_name=cluster_name, - dataproc_properties=dataproc_properties, - dataproc_jars=dataproc_jars, - **kwargs, - ) - self.query = query - self.query_uri = query_uri - self.variables = variables - if self.query is not None and self.query_uri is not None: - raise AirflowException("Only one of `query` and `query_uri` can be passed.") - - def generate_job(self): - """ - Act as a helper method for easier migration to `DataprocSubmitJobOperator`. - - :return: Dict representing Dataproc job - """ - job_template = self.create_job_template() - if self.query is None: - if self.query_uri is None: - raise AirflowException("One of query or query_uri should be set here") - job_template.add_query_uri(self.query_uri) - else: - job_template.add_query(self.query) - job_template.add_variables(self.variables) - return self._generate_job_template() - - def execute(self, context: Context): - job_template = self.create_job_template() - if self.query is None: - if self.query_uri is None: - raise AirflowException("One of query or query_uri should be set here") - job_template.add_query_uri(self.query_uri) - else: - job_template.add_query(self.query) - job_template.add_variables(self.variables) - super().execute(context) - - -# TODO: Remove one day -@deprecated( - planned_removal_date="November 01, 2024", - use_instead="DataprocSubmitJobOperator", - instructions="You can use `generate_job` method to generate dictionary representing your job " - "and use it with the new operator.", - category=AirflowProviderDeprecationWarning, -) -class DataprocSubmitSparkSqlJobOperator(DataprocJobBaseOperator): - """ - Start a Spark SQL query Job on a Cloud DataProc cluster. - - .. seealso:: - This operator is deprecated, please use - :class:`~airflow.providers.google.cloud.operators.dataproc.DataprocSubmitJobOperator`: - - :param query: The query or reference to the query file (q extension). (templated) - :param query_uri: The HCFS URI of the script that contains the SQL queries. - :param variables: Map of named parameters for the query. (templated) - """ - - template_fields: Sequence[str] = ( - "query", - "variables", - "job_name", - "cluster_name", - "region", - "dataproc_jars", - "dataproc_properties", - "impersonation_chain", - ) - template_ext = (".q",) - template_fields_renderers = {"sql": "sql"} - ui_color = "#0273d4" - job_type = "spark_sql_job" - - def __init__( - self, - *, - query: str | None = None, - query_uri: str | None = None, - variables: dict | None = None, - impersonation_chain: str | Sequence[str] | None = None, - region: str, - job_name: str = "{{task.task_id}}_{{ds_nodash}}", - cluster_name: str = "cluster-1", - dataproc_properties: dict | None = None, - dataproc_jars: list[str] | None = None, - **kwargs, - ) -> None: - super().__init__( - impersonation_chain=impersonation_chain, - region=region, - job_name=job_name, - cluster_name=cluster_name, - dataproc_properties=dataproc_properties, - dataproc_jars=dataproc_jars, - **kwargs, - ) - self.query = query - self.query_uri = query_uri - self.variables = variables - if self.query is not None and self.query_uri is not None: - raise AirflowException("Only one of `query` and `query_uri` can be passed.") - - def generate_job(self): - """ - Act as a helper method for easier migration to `DataprocSubmitJobOperator`. - - :return: Dict representing Dataproc job - """ - job_template = self.create_job_template() - if self.query is None: - job_template.add_query_uri(self.query_uri) - else: - job_template.add_query(self.query) - job_template.add_variables(self.variables) - return self._generate_job_template() - - def execute(self, context: Context): - job_template = self.create_job_template() - if self.query is None: - if self.query_uri is None: - raise AirflowException("One of query or query_uri should be set here") - job_template.add_query_uri(self.query_uri) - else: - job_template.add_query(self.query) - job_template.add_variables(self.variables) - super().execute(context) - - -# TODO: Remove one day -@deprecated( - planned_removal_date="November 01, 2024", - use_instead="DataprocSubmitJobOperator", - instructions="You can use `generate_job` method to generate dictionary representing your job " - "and use it with the new operator.", - category=AirflowProviderDeprecationWarning, -) -class DataprocSubmitSparkJobOperator(DataprocJobBaseOperator): - """ - Start a Spark Job on a Cloud DataProc cluster. - - .. seealso:: - This operator is deprecated, please use - :class:`~airflow.providers.google.cloud.operators.dataproc.DataprocSubmitJobOperator`: - - :param main_jar: The HCFS URI of the jar file that contains the main class - (use this or the main_class, not both together). - :param main_class: Name of the job class. (use this or the main_jar, not both - together). - :param arguments: Arguments for the job. (templated) - :param archives: List of archived files that will be unpacked in the work - directory. Should be stored in Cloud Storage. - :param files: List of files to be copied to the working directory - """ - - template_fields: Sequence[str] = ( - "arguments", - "job_name", - "cluster_name", - "region", - "dataproc_jars", - "dataproc_properties", - "impersonation_chain", - ) - ui_color = "#0273d4" - job_type = "spark_job" - - def __init__( - self, - *, - main_jar: str | None = None, - main_class: str | None = None, - arguments: list | None = None, - archives: list | None = None, - files: list | None = None, - impersonation_chain: str | Sequence[str] | None = None, - region: str, - job_name: str = "{{task.task_id}}_{{ds_nodash}}", - cluster_name: str = "cluster-1", - dataproc_properties: dict | None = None, - dataproc_jars: list[str] | None = None, - **kwargs, - ) -> None: - super().__init__( - impersonation_chain=impersonation_chain, - region=region, - job_name=job_name, - cluster_name=cluster_name, - dataproc_properties=dataproc_properties, - dataproc_jars=dataproc_jars, - **kwargs, - ) - self.main_jar = main_jar - self.main_class = main_class - self.arguments = arguments - self.archives = archives - self.files = files - - def generate_job(self): - """ - Act as a helper method for easier migration to `DataprocSubmitJobOperator`. - - :return: Dict representing Dataproc job - """ - job_template = self.create_job_template() - job_template.set_main(self.main_jar, self.main_class) - job_template.add_args(self.arguments) - job_template.add_archive_uris(self.archives) - job_template.add_file_uris(self.files) - return self._generate_job_template() - - def execute(self, context: Context): - job_template = self.create_job_template() - job_template.set_main(self.main_jar, self.main_class) - job_template.add_args(self.arguments) - job_template.add_archive_uris(self.archives) - job_template.add_file_uris(self.files) - super().execute(context) - - -# TODO: Remove one day -@deprecated( - planned_removal_date="November 01, 2024", - use_instead="DataprocSubmitJobOperator", - instructions="You can use `generate_job` method to generate dictionary representing your job " - "and use it with the new operator.", - category=AirflowProviderDeprecationWarning, -) -class DataprocSubmitHadoopJobOperator(DataprocJobBaseOperator): - """ - Start a Hadoop Job on a Cloud DataProc cluster. - - .. seealso:: - This operator is deprecated, please use - :class:`~airflow.providers.google.cloud.operators.dataproc.DataprocSubmitJobOperator`: - - :param main_jar: The HCFS URI of the jar file containing the main class - (use this or the main_class, not both together). - :param main_class: Name of the job class. (use this or the main_jar, not both - together). - :param arguments: Arguments for the job. (templated) - :param archives: List of archived files that will be unpacked in the work - directory. Should be stored in Cloud Storage. - :param files: List of files to be copied to the working directory - """ - - template_fields: Sequence[str] = ( - "arguments", - "job_name", - "cluster_name", - "region", - "dataproc_jars", - "dataproc_properties", - "impersonation_chain", - ) - ui_color = "#0273d4" - job_type = "hadoop_job" - - def __init__( - self, - *, - main_jar: str | None = None, - main_class: str | None = None, - arguments: list | None = None, - archives: list | None = None, - files: list | None = None, - impersonation_chain: str | Sequence[str] | None = None, - region: str, - job_name: str = "{{task.task_id}}_{{ds_nodash}}", - cluster_name: str = "cluster-1", - dataproc_properties: dict | None = None, - dataproc_jars: list[str] | None = None, - **kwargs, - ) -> None: - super().__init__( - impersonation_chain=impersonation_chain, - region=region, - job_name=job_name, - cluster_name=cluster_name, - dataproc_properties=dataproc_properties, - dataproc_jars=dataproc_jars, - **kwargs, - ) - self.main_jar = main_jar - self.main_class = main_class - self.arguments = arguments - self.archives = archives - self.files = files - - def generate_job(self): - """ - Act as a helper method for easier migration to `DataprocSubmitJobOperator`. - - :return: Dict representing Dataproc job - """ - job_template = self.create_job_template() - job_template.set_main(self.main_jar, self.main_class) - job_template.add_args(self.arguments) - job_template.add_archive_uris(self.archives) - job_template.add_file_uris(self.files) - return self._generate_job_template() - - def execute(self, context: Context): - job_template = self.create_job_template() - job_template.set_main(self.main_jar, self.main_class) - job_template.add_args(self.arguments) - job_template.add_archive_uris(self.archives) - job_template.add_file_uris(self.files) - super().execute(context) - - -# TODO: Remove one day -@deprecated( - planned_removal_date="November 01, 2024", - use_instead="DataprocSubmitJobOperator", - instructions="You can use `generate_job` method to generate dictionary representing your job " - "and use it with the new operator.", - category=AirflowProviderDeprecationWarning, -) -class DataprocSubmitPySparkJobOperator(DataprocJobBaseOperator): - """ - Start a PySpark Job on a Cloud DataProc cluster. - - .. seealso:: - This operator is deprecated, please use - :class:`~airflow.providers.google.cloud.operators.dataproc.DataprocSubmitJobOperator`: - - :param main: [Required] The Hadoop Compatible Filesystem (HCFS) URI of the main - Python file to use as the driver. Must be a .py file. (templated) - :param arguments: Arguments for the job. (templated) - :param archives: List of archived files that will be unpacked in the work - directory. Should be stored in Cloud Storage. - :param files: List of files to be copied to the working directory - :param pyfiles: List of Python files to pass to the PySpark framework. - Supported file types: .py, .egg, and .zip - """ - - template_fields: Sequence[str] = ( - "main", - "arguments", - "job_name", - "cluster_name", - "region", - "dataproc_jars", - "dataproc_properties", - "impersonation_chain", - ) - ui_color = "#0273d4" - job_type = "pyspark_job" - - @staticmethod - def _generate_temp_filename(filename): - return f"{time.strftime('%Y%m%d%H%M%S')}_{uuid.uuid4()!s:.8}_{ntpath.basename(filename)}" - - def _upload_file_temp(self, bucket, local_file): - """Upload a local file to a Google Cloud Storage bucket.""" - temp_filename = self._generate_temp_filename(local_file) - if not bucket: - raise AirflowException( - "If you want Airflow to upload the local file to a temporary bucket, set " - "the 'temp_bucket' key in the connection string" - ) - - self.log.info("Uploading %s to %s", local_file, temp_filename) - - GCSHook(gcp_conn_id=self.gcp_conn_id, impersonation_chain=self.impersonation_chain).upload( - bucket_name=bucket, - object_name=temp_filename, - mime_type="application/x-python", - filename=local_file, - ) - return f"gs://{bucket}/{temp_filename}" - - def __init__( - self, - *, - main: str, - arguments: list | None = None, - archives: list | None = None, - pyfiles: list | None = None, - files: list | None = None, - impersonation_chain: str | Sequence[str] | None = None, - region: str, - job_name: str = "{{task.task_id}}_{{ds_nodash}}", - cluster_name: str = "cluster-1", - dataproc_properties: dict | None = None, - dataproc_jars: list[str] | None = None, - **kwargs, - ) -> None: - super().__init__( - impersonation_chain=impersonation_chain, - region=region, - job_name=job_name, - cluster_name=cluster_name, - dataproc_properties=dataproc_properties, - dataproc_jars=dataproc_jars, - **kwargs, - ) - self.main = main - self.arguments = arguments - self.archives = archives - self.files = files - self.pyfiles = pyfiles - - def generate_job(self): - """ - Act as a helper method for easier migration to :class:`DataprocSubmitJobOperator`. - - :return: Dict representing Dataproc job - """ - job_template = self.create_job_template() - # Check if the file is local, if that is the case, upload it to a bucket - if os.path.isfile(self.main): - cluster_info = self.hook.get_cluster( - project_id=self.project_id, region=self.region, cluster_name=self.cluster_name - ) - bucket = cluster_info["config"]["config_bucket"] - self.main = f"gs://{bucket}/{self.main}" - job_template.set_python_main(self.main) - job_template.add_args(self.arguments) - job_template.add_archive_uris(self.archives) - job_template.add_file_uris(self.files) - job_template.add_python_file_uris(self.pyfiles) - - return self._generate_job_template() - - def execute(self, context: Context): - job_template = self.create_job_template() - # Check if the file is local, if that is the case, upload it to a bucket - if os.path.isfile(self.main): - cluster_info = self.hook.get_cluster( - project_id=self.project_id, region=self.region, cluster_name=self.cluster_name - ) - bucket = cluster_info["config"]["config_bucket"] - self.main = self._upload_file_temp(bucket, self.main) - - job_template.set_python_main(self.main) - job_template.add_args(self.arguments) - job_template.add_archive_uris(self.archives) - job_template.add_file_uris(self.files) - job_template.add_python_file_uris(self.pyfiles) - super().execute(context) - - class DataprocCreateWorkflowTemplateOperator(GoogleCloudBaseOperator): """ Creates new workflow template. diff --git a/providers/src/airflow/providers/google/cloud/operators/kubernetes_engine.py b/providers/src/airflow/providers/google/cloud/operators/kubernetes_engine.py index c19bc4ff5085..84de9cfdafcb 100644 --- a/providers/src/airflow/providers/google/cloud/operators/kubernetes_engine.py +++ b/providers/src/airflow/providers/google/cloud/operators/kubernetes_engine.py @@ -56,7 +56,6 @@ GKEOperationTrigger, GKEStartPodTrigger, ) -from airflow.providers.google.common.deprecated import deprecated from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID from airflow.providers_manager import ProvidersManager from airflow.utils.timezone import utcnow @@ -723,15 +722,6 @@ def __init__( if self.config_file: raise AirflowException("config_file is not an allowed parameter for the GKEStartPodOperator.") - @staticmethod - @deprecated( - planned_removal_date="November 01, 2024", - use_instead="fetch_cluster_info", - category=AirflowProviderDeprecationWarning, - ) - def get_gke_config_file(): - pass - @cached_property def cluster_hook(self) -> GKEHook: return GKEHook( diff --git a/providers/src/airflow/providers/google/cloud/operators/vertex_ai/auto_ml.py b/providers/src/airflow/providers/google/cloud/operators/vertex_ai/auto_ml.py index 86b3ae017c08..d3069984a830 100644 --- a/providers/src/airflow/providers/google/cloud/operators/vertex_ai/auto_ml.py +++ b/providers/src/airflow/providers/google/cloud/operators/vertex_ai/auto_ml.py @@ -464,98 +464,6 @@ def execute(self, context: Context): return result -@deprecated( - planned_removal_date="September 15, 2024", - use_instead="SupervisedFineTuningTrainOperator", - instructions=( - "Please consider using Fine Tuning over the Gemini model. " - "More info: https://cloud.google.com/vertex-ai/docs/start/automl-gemini-comparison" - ), - category=AirflowProviderDeprecationWarning, -) -class CreateAutoMLTextTrainingJobOperator(AutoMLTrainingJobBaseOperator): - """ - Create Auto ML Text Training job. - - WARNING: Text creation API is deprecated since September 15, 2024 - (https://cloud.google.com/vertex-ai/docs/tutorials/text-classification-automl/overview). - """ - - template_fields = [ - "parent_model", - "dataset_id", - "region", - "impersonation_chain", - ] - operator_extra_links = (VertexAIModelLink(), VertexAITrainingLink()) - - def __init__( - self, - *, - dataset_id: str, - prediction_type: str, - multi_label: bool = False, - sentiment_max: int = 10, - validation_fraction_split: float | None = None, - training_filter_split: str | None = None, - validation_filter_split: str | None = None, - test_filter_split: str | None = None, - **kwargs, - ) -> None: - super().__init__(**kwargs) - self.dataset_id = dataset_id - self.prediction_type = prediction_type - self.multi_label = multi_label - self.sentiment_max = sentiment_max - self.validation_fraction_split = validation_fraction_split - self.training_filter_split = training_filter_split - self.validation_filter_split = validation_filter_split - self.test_filter_split = test_filter_split - - def execute(self, context: Context): - self.hook = AutoMLHook( - gcp_conn_id=self.gcp_conn_id, - impersonation_chain=self.impersonation_chain, - ) - self.parent_model = self.parent_model.split("@")[0] if self.parent_model else None - model, training_id = self.hook.create_auto_ml_text_training_job( - project_id=self.project_id, - region=self.region, - display_name=self.display_name, - dataset=datasets.TextDataset(dataset_name=self.dataset_id), - prediction_type=self.prediction_type, - multi_label=self.multi_label, - sentiment_max=self.sentiment_max, - labels=self.labels, - training_encryption_spec_key_name=self.training_encryption_spec_key_name, - model_encryption_spec_key_name=self.model_encryption_spec_key_name, - training_fraction_split=self.training_fraction_split, - validation_fraction_split=self.validation_fraction_split, - test_fraction_split=self.test_fraction_split, - training_filter_split=self.training_filter_split, - validation_filter_split=self.validation_filter_split, - test_filter_split=self.test_filter_split, - model_display_name=self.model_display_name, - model_labels=self.model_labels, - sync=self.sync, - parent_model=self.parent_model, - is_default_version=self.is_default_version, - model_version_aliases=self.model_version_aliases, - model_version_description=self.model_version_description, - ) - - if model: - result = Model.to_dict(model) - model_id = self.hook.extract_model_id(result) - self.xcom_push(context, key="model_id", value=model_id) - VertexAIModelLink.persist(context=context, task_instance=self, model_id=model_id) - else: - result = model # type: ignore - self.xcom_push(context, key="training_id", value=training_id) - VertexAITrainingLink.persist(context=context, task_instance=self, training_id=training_id) - return result - - class CreateAutoMLVideoTrainingJobOperator(AutoMLTrainingJobBaseOperator): """Create Auto ML Video Training job.""" diff --git a/providers/src/airflow/providers/google/cloud/operators/vertex_ai/batch_prediction_job.py b/providers/src/airflow/providers/google/cloud/operators/vertex_ai/batch_prediction_job.py index 60ddd747eec5..97f5b508e752 100644 --- a/providers/src/airflow/providers/google/cloud/operators/vertex_ai/batch_prediction_job.py +++ b/providers/src/airflow/providers/google/cloud/operators/vertex_ai/batch_prediction_job.py @@ -20,7 +20,6 @@ from __future__ import annotations -import warnings from functools import cached_property from typing import TYPE_CHECKING, Any, Sequence @@ -29,7 +28,7 @@ from google.cloud.aiplatform_v1.types import BatchPredictionJob from airflow.configuration import conf -from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning +from airflow.exceptions import AirflowException from airflow.providers.google.cloud.hooks.vertex_ai.batch_prediction_job import BatchPredictionJobHook from airflow.providers.google.cloud.links.vertex_ai import ( VertexAIBatchPredictionJobLink, @@ -136,9 +135,6 @@ class CreateBatchPredictionJobOperator(GoogleCloudBaseOperator): If this is set, then all resources created by the BatchPredictionJob will be encrypted with the provided encryption key. Overrides encryption_spec_key_name set in aiplatform.init. - :param sync: (Deprecated) Whether to execute this method synchronously. If False, this method will be executed in - concurrent Future and any downstream object will be immediately returned and synced when the - Future has completed. :param create_request_timeout: Optional. The timeout for the create request in seconds. :param batch_size: Optional. The number of the records (e.g. instances) of the operation given in each batch @@ -190,7 +186,6 @@ def __init__( explanation_parameters: explain.ExplanationParameters | None = None, labels: dict[str, str] | None = None, encryption_spec_key_name: str | None = None, - sync: bool = True, create_request_timeout: float | None = None, batch_size: int | None = None, gcp_conn_id: str = "google_cloud_default", @@ -221,7 +216,6 @@ def __init__( self.explanation_parameters = explanation_parameters self.labels = labels self.encryption_spec_key_name = encryption_spec_key_name - self.sync = sync self.create_request_timeout = create_request_timeout self.batch_size = batch_size self.gcp_conn_id = gcp_conn_id @@ -237,11 +231,6 @@ def hook(self) -> BatchPredictionJobHook: ) def execute(self, context: Context): - warnings.warn( - "The 'sync' parameter is deprecated and will be removed after 28.08.2024.", - AirflowProviderDeprecationWarning, - stacklevel=2, - ) self.log.info("Creating Batch prediction job") batch_prediction_job: BatchPredictionJobObject = self.hook.submit_batch_prediction_job( region=self.region, diff --git a/providers/src/airflow/providers/google/cloud/operators/vertex_ai/custom_job.py b/providers/src/airflow/providers/google/cloud/operators/vertex_ai/custom_job.py index 57cb7759a4a7..ebde5c24597a 100644 --- a/providers/src/airflow/providers/google/cloud/operators/vertex_ai/custom_job.py +++ b/providers/src/airflow/providers/google/cloud/operators/vertex_ai/custom_job.py @@ -19,7 +19,6 @@ from __future__ import annotations -import warnings from functools import cached_property from typing import TYPE_CHECKING, Any, Sequence @@ -111,7 +110,6 @@ def __init__( predefined_split_column_name: str | None = None, timestamp_split_column_name: str | None = None, tensorboard: str | None = None, - sync=True, gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, **kwargs, @@ -168,18 +166,10 @@ def __init__( self.predefined_split_column_name = predefined_split_column_name self.timestamp_split_column_name = timestamp_split_column_name self.tensorboard = tensorboard - self.sync = sync # END Run param self.gcp_conn_id = gcp_conn_id self.impersonation_chain = impersonation_chain - def execute(self, context: Context) -> None: - warnings.warn( - "The 'sync' parameter is deprecated and will be removed after 01.10.2024.", - AirflowProviderDeprecationWarning, - stacklevel=2, - ) - def execute_complete(self, context: Context, event: dict[str, Any]) -> dict[str, Any] | None: if event["status"] == "error": raise AirflowException(event["message"]) @@ -532,8 +522,6 @@ def __init__( self.poll_interval = poll_interval def execute(self, context: Context): - super().execute(context) - self.parent_model = self.parent_model.split("@")[0] if self.parent_model else None if self.deferrable: @@ -990,8 +978,6 @@ def __init__( self.poll_interval = poll_interval def execute(self, context: Context): - super().execute(context) - self.parent_model = self.parent_model.split("@")[0] if self.parent_model else None if self.deferrable: @@ -1455,8 +1441,6 @@ def __init__( self.poll_interval = poll_interval def execute(self, context: Context): - super().execute(context) - self.parent_model = self.parent_model.split("@")[0] if self.parent_model else None if self.deferrable: diff --git a/providers/src/airflow/providers/google/cloud/operators/vertex_ai/hyperparameter_tuning_job.py b/providers/src/airflow/providers/google/cloud/operators/vertex_ai/hyperparameter_tuning_job.py index 43229e033cc8..4c3ce251f06b 100644 --- a/providers/src/airflow/providers/google/cloud/operators/vertex_ai/hyperparameter_tuning_job.py +++ b/providers/src/airflow/providers/google/cloud/operators/vertex_ai/hyperparameter_tuning_job.py @@ -20,7 +20,6 @@ from __future__ import annotations -import warnings from typing import TYPE_CHECKING, Any, Sequence from google.api_core.exceptions import NotFound @@ -28,7 +27,7 @@ from google.cloud.aiplatform_v1 import types from airflow.configuration import conf -from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning +from airflow.exceptions import AirflowException from airflow.providers.google.cloud.hooks.vertex_ai.hyperparameter_tuning_job import ( HyperparameterTuningJobHook, ) @@ -128,8 +127,6 @@ class CreateHyperparameterTuningJobOperator(GoogleCloudBaseOperator): `service_account` is required with provided `tensorboard`. For more information on configuring your service account please visit: https://cloud.google.com/vertex-ai/docs/experiments/tensorboard-training - :param sync: (Deprecated) Whether to execute this method synchronously. If False, this method will - unblock, and it will be executed in a concurrent Future. :param gcp_conn_id: The connection ID to use connecting to Google Cloud. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token @@ -180,7 +177,6 @@ def __init__( restart_job_on_worker_restart: bool = False, enable_web_access: bool = False, tensorboard: str | None = None, - sync: bool = True, # END: run param gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, @@ -214,7 +210,6 @@ def __init__( self.restart_job_on_worker_restart = restart_job_on_worker_restart self.enable_web_access = enable_web_access self.tensorboard = tensorboard - self.sync = sync self.gcp_conn_id = gcp_conn_id self.impersonation_chain = impersonation_chain self.hook: HyperparameterTuningJobHook | None = None @@ -222,12 +217,6 @@ def __init__( self.poll_interval = poll_interval def execute(self, context: Context): - warnings.warn( - "The 'sync' parameter is deprecated and will be removed after 01.09.2024.", - AirflowProviderDeprecationWarning, - stacklevel=2, - ) - self.log.info("Creating Hyperparameter Tuning job") self.hook = HyperparameterTuningJobHook( gcp_conn_id=self.gcp_conn_id, diff --git a/providers/src/airflow/providers/google/cloud/secrets/secret_manager.py b/providers/src/airflow/providers/google/cloud/secrets/secret_manager.py index dde659020a82..b0f7ba523377 100644 --- a/providers/src/airflow/providers/google/cloud/secrets/secret_manager.py +++ b/providers/src/airflow/providers/google/cloud/secrets/secret_manager.py @@ -23,13 +23,12 @@ from google.auth.exceptions import DefaultCredentialsError -from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning +from airflow.exceptions import AirflowException from airflow.providers.google.cloud._internal_client.secret_manager_client import _SecretManagerClient from airflow.providers.google.cloud.utils.credentials_provider import ( _get_target_principal_and_delegates, get_credentials_and_project_id, ) -from airflow.providers.google.common.deprecated import deprecated from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID from airflow.secrets import BaseSecretsBackend from airflow.utils.log.logging_mixin import LoggingMixin @@ -161,22 +160,6 @@ def get_conn_value(self, conn_id: str) -> str | None: return self._get_secret(self.connections_prefix, conn_id) - @deprecated( - planned_removal_date="November 01, 2024", - use_instead="get_conn_value", - category=AirflowProviderDeprecationWarning, - ) - def get_conn_uri(self, conn_id: str) -> str | None: - """ - Return URI representation of Connection conn_id. - - As of Airflow version 2.3.0 this method is deprecated. - - :param conn_id: the connection id - :return: deserialized Connection - """ - return self.get_conn_value(conn_id) - def get_variable(self, key: str) -> str | None: """ Get Airflow Variable from Environment Variable. diff --git a/providers/src/airflow/providers/google/cloud/sensors/bigquery.py b/providers/src/airflow/providers/google/cloud/sensors/bigquery.py index 7a866c7b0ac6..c63ad5ca7a91 100644 --- a/providers/src/airflow/providers/google/cloud/sensors/bigquery.py +++ b/providers/src/airflow/providers/google/cloud/sensors/bigquery.py @@ -30,7 +30,6 @@ BigQueryTableExistenceTrigger, BigQueryTablePartitionExistenceTrigger, ) -from airflow.providers.google.common.deprecated import deprecated from airflow.sensors.base import BaseSensorOperator if TYPE_CHECKING: @@ -257,82 +256,3 @@ def execute_complete(self, context: dict[str, Any], event: dict[str, str] | None message = "No event received in trigger callback" raise AirflowException(message) - - -@deprecated( - planned_removal_date="November 01, 2024", - use_instead="BigQueryTableExistenceSensor", - instructions="Please use BigQueryTableExistenceSensor and set deferrable attribute to True.", - category=AirflowProviderDeprecationWarning, -) -class BigQueryTableExistenceAsyncSensor(BigQueryTableExistenceSensor): - """ - Checks for the existence of a table in Google Big Query. - - This class is deprecated and will be removed in a future release. - - Please use :class:`airflow.providers.google.cloud.sensors.bigquery.BigQueryTableExistenceSensor` - and set *deferrable* attribute to *True* instead. - - :param project_id: The Google cloud project in which to look for the table. - The connection supplied to the hook must provide - access to the specified project. - :param dataset_id: The name of the dataset in which to look for the table. - storage bucket. - :param table_id: The name of the table to check the existence of. - :param gcp_conn_id: The connection ID used to connect to Google Cloud. - :param bigquery_conn_id: (Deprecated) The connection ID used to connect to Google Cloud. - This parameter has been deprecated. You should pass the gcp_conn_id parameter instead. - :param impersonation_chain: Optional service account to impersonate using short-term - credentials, or chained list of accounts required to get the access_token - of the last account in the list, which will be impersonated in the request. - If set as a string, the account must grant the originating account - the Service Account Token Creator IAM role. - If set as a sequence, the identities from the list must grant - Service Account Token Creator IAM role to the directly preceding identity, with first - account from the list granting this role to the originating account (templated). - :param polling_interval: The interval in seconds to wait between checks table existence. - """ - - def __init__(self, **kwargs): - super().__init__(deferrable=True, **kwargs) - - -@deprecated( - planned_removal_date="November 01, 2024", - use_instead="BigQueryTablePartitionExistenceSensor", - instructions="Please use BigQueryTablePartitionExistenceSensor and set deferrable attribute to True.", - category=AirflowProviderDeprecationWarning, -) -class BigQueryTableExistencePartitionAsyncSensor(BigQueryTablePartitionExistenceSensor): - """ - Checks for the existence of a partition within a table in Google BigQuery. - - This class is deprecated and will be removed in a future release. - - Please use :class:`airflow.providers.google.cloud.sensors.bigquery.BigQueryTablePartitionExistenceSensor` - and set *deferrable* attribute to *True* instead. - - :param project_id: The Google cloud project in which to look for the table. - The connection supplied to the hook must provide - access to the specified project. - :param dataset_id: The name of the dataset in which to look for the table. - storage bucket. - :param partition_id: The name of the partition to check the existence of. - :param table_id: The name of the table to check the existence of. - :param gcp_conn_id: The connection ID used to connect to Google Cloud. - :param bigquery_conn_id: (Deprecated) The connection ID used to connect to Google Cloud. - This parameter has been deprecated. You should pass the gcp_conn_id parameter instead. - :param impersonation_chain: Optional service account to impersonate using short-term - credentials, or chained list of accounts required to get the access_token - of the last account in the list, which will be impersonated in the request. - If set as a string, the account must grant the originating account - the Service Account Token Creator IAM role. - If set as a sequence, the identities from the list must grant - Service Account Token Creator IAM role to the directly preceding identity, with first - account from the list granting this role to the originating account (templated). - :param poke_interval: The interval in seconds to wait between checks table existence. - """ - - def __init__(self, **kwargs): - super().__init__(deferrable=True, **kwargs) diff --git a/providers/src/airflow/providers/google/cloud/sensors/cloud_composer.py b/providers/src/airflow/providers/google/cloud/sensors/cloud_composer.py index 40703c02fe2a..e977340ede9e 100644 --- a/providers/src/airflow/providers/google/cloud/sensors/cloud_composer.py +++ b/providers/src/airflow/providers/google/cloud/sensors/cloud_composer.py @@ -21,20 +21,16 @@ import json from datetime import datetime, timedelta -from typing import TYPE_CHECKING, Any, Iterable, Sequence +from typing import TYPE_CHECKING, Iterable, Sequence from dateutil import parser from google.cloud.orchestration.airflow.service_v1.types import ExecuteAirflowCommandResponse from airflow.configuration import conf -from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning +from airflow.exceptions import AirflowException from airflow.providers.google.cloud.hooks.cloud_composer import CloudComposerHook -from airflow.providers.google.cloud.triggers.cloud_composer import ( - CloudComposerDAGRunTrigger, - CloudComposerExecutionTrigger, -) +from airflow.providers.google.cloud.triggers.cloud_composer import CloudComposerDAGRunTrigger from airflow.providers.google.common.consts import GOOGLE_DEFAULT_DEFERRABLE_METHOD_NAME -from airflow.providers.google.common.deprecated import deprecated from airflow.sensors.base import BaseSensorOperator from airflow.utils.state import TaskInstanceState @@ -42,89 +38,6 @@ from airflow.utils.context import Context -@deprecated( - planned_removal_date="November 01, 2024", - use_instead="CloudComposerCreateEnvironmentOperator, CloudComposerDeleteEnvironmentOperator, " - "CloudComposerUpdateEnvironmentOperator", - instructions="Please use CloudComposerCreateEnvironmentOperator, CloudComposerDeleteEnvironmentOperator " - "or CloudComposerUpdateEnvironmentOperator in deferrable or non-deferrable mode, " - "since since every operator gives user a possibility to wait (asynchronously or synchronously) " - "until the Operation is finished.", - category=AirflowProviderDeprecationWarning, -) -class CloudComposerEnvironmentSensor(BaseSensorOperator): - """ - Check the status of the Cloud Composer Environment task. - - This Sensor is deprecated. You can achieve the same functionality by using Cloud Composer Operators - CloudComposerCreateEnvironmentOperator, CloudComposerDeleteEnvironmentOperator and - CloudComposerUpdateEnvironmentOperator in deferrable or non-deferrable mode, since every operator - gives user a possibility to wait (asynchronously or synchronously) until Operation will be finished. - - :param project_id: Required. The ID of the Google Cloud project that the service belongs to. - :param region: Required. The ID of the Google Cloud region that the service belongs to. - :param operation_name: The name of the operation resource - :param gcp_conn_id: The connection ID to use when fetching connection info. - :param impersonation_chain: Optional service account to impersonate using short-term - credentials, or chained list of accounts required to get the access_token - of the last account in the list, which will be impersonated in the request. - If set as a string, the account must grant the originating account - the Service Account Token Creator IAM role. - If set as a sequence, the identities from the list must grant - Service Account Token Creator IAM role to the directly preceding identity, with first - account from the list granting this role to the originating account (templated). - :param pooling_period_seconds: Optional: Control the rate of the poll for the result of deferrable run. - """ - - def __init__( - self, - *, - project_id: str, - region: str, - operation_name: str, - gcp_conn_id: str = "google_cloud_default", - impersonation_chain: str | Sequence[str] | None = None, - pooling_period_seconds: int = 30, - **kwargs, - ): - super().__init__(**kwargs) - self.project_id = project_id - self.region = region - self.operation_name = operation_name - self.pooling_period_seconds = pooling_period_seconds - self.gcp_conn_id = gcp_conn_id - self.impersonation_chain = impersonation_chain - - def execute(self, context: Context) -> None: - """Airflow runs this method on the worker and defers using the trigger.""" - self.defer( - trigger=CloudComposerExecutionTrigger( - project_id=self.project_id, - region=self.region, - operation_name=self.operation_name, - gcp_conn_id=self.gcp_conn_id, - impersonation_chain=self.impersonation_chain, - pooling_period_seconds=self.pooling_period_seconds, - ), - method_name="execute_complete", - ) - - def execute_complete(self, context: dict[str, Any], event: dict[str, str] | None = None) -> str: - """ - Act as a callback for when the trigger fires - returns immediately. - - Relies on trigger to throw an exception, otherwise it assumes execution was successful. - """ - if event: - if event.get("operation_done"): - return event["operation_done"] - - raise AirflowException(event["message"]) - - message = "No event received in trigger callback" - raise AirflowException(message) - - class CloudComposerDAGRunSensor(BaseSensorOperator): """ Check if a DAG run has completed. diff --git a/providers/src/airflow/providers/google/cloud/sensors/gcs.py b/providers/src/airflow/providers/google/cloud/sensors/gcs.py index 2b5782c77ad4..8493a0facb66 100644 --- a/providers/src/airflow/providers/google/cloud/sensors/gcs.py +++ b/providers/src/airflow/providers/google/cloud/sensors/gcs.py @@ -27,7 +27,7 @@ from google.cloud.storage.retry import DEFAULT_RETRY from airflow.configuration import conf -from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning +from airflow.exceptions import AirflowException from airflow.providers.google.cloud.hooks.gcs import GCSHook from airflow.providers.google.cloud.triggers.gcs import ( GCSBlobTrigger, @@ -35,7 +35,6 @@ GCSPrefixBlobTrigger, GCSUploadSessionTrigger, ) -from airflow.providers.google.common.deprecated import deprecated from airflow.sensors.base import BaseSensorOperator, poke_mode_only if TYPE_CHECKING: @@ -142,38 +141,6 @@ def execute_complete(self, context: Context, event: dict[str, str]) -> bool: return True -@deprecated( - planned_removal_date="November 01, 2024", - use_instead="GCSObjectExistenceSensor", - instructions="Please use GCSObjectExistenceSensor and set deferrable attribute to True.", - category=AirflowProviderDeprecationWarning, -) -class GCSObjectExistenceAsyncSensor(GCSObjectExistenceSensor): - """ - Checks for the existence of a file in Google Cloud Storage. - - This class is deprecated and will be removed in a future release. - - Please use :class:`airflow.providers.google.cloud.sensors.gcs.GCSObjectExistenceSensor` - and set *deferrable* attribute to *True* instead. - - :param bucket: The Google Cloud Storage bucket where the object is. - :param object: The name of the object to check in the Google cloud storage bucket. - :param google_cloud_conn_id: The connection ID to use when connecting to Google Cloud Storage. - :param impersonation_chain: Optional service account to impersonate using short-term - credentials, or chained list of accounts required to get the access_token - of the last account in the list, which will be impersonated in the request. - If set as a string, the account must grant the originating account - the Service Account Token Creator IAM role. - If set as a sequence, the identities from the list must grant - Service Account Token Creator IAM role to the directly preceding identity, with first - account from the list granting this role to the originating account (templated). - """ - - def __init__(self, **kwargs: Any) -> None: - super().__init__(deferrable=True, **kwargs) - - def ts_function(context): """ Act as a default callback for the GoogleCloudStorageObjectUpdatedSensor. diff --git a/providers/src/airflow/providers/google/cloud/triggers/gcs.py b/providers/src/airflow/providers/google/cloud/triggers/gcs.py index 3ec1bbddfd73..9054260ee5ba 100644 --- a/providers/src/airflow/providers/google/cloud/triggers/gcs.py +++ b/providers/src/airflow/providers/google/cloud/triggers/gcs.py @@ -122,7 +122,7 @@ class GCSCheckBlobUpdateTimeTrigger(BaseTrigger): :param target_date: context datetime to compare with blob object updated time :param poke_interval: polling period in seconds to check for file/folder :param google_cloud_conn_id: reference to the Google Connection - :param hook_params: dict object that has delegate_to and impersonation_chain + :param hook_params: dict object that has impersonation_chain """ def __init__( diff --git a/providers/src/airflow/providers/google/common/hooks/base_google.py b/providers/src/airflow/providers/google/common/hooks/base_google.py index c55ac27aa0ec..56795e3a17b1 100644 --- a/providers/src/airflow/providers/google/common/hooks/base_google.py +++ b/providers/src/airflow/providers/google/common/hooks/base_google.py @@ -197,12 +197,6 @@ class GoogleBaseHook(BaseHook): JSON data provided in the UI: Specify 'Keyfile JSON'. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. The usage of this parameter should be limited only to Google Workspace - (gsuite) and marketing platform operators and hooks. It is deprecated for usage by Google Cloud - and Firebase operators and hooks, as well as transfer operators in other providers that involve - Google cloud. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -279,12 +273,11 @@ def get_ui_field_behaviour(cls) -> dict[str, Any]: def __init__( self, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: - super().__init__() + super().__init__(**kwargs) self.gcp_conn_id = gcp_conn_id - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain self.extras: dict = self.get_connection(self.gcp_conn_id).extra_dejson self._cached_credentials: Credentials | None = None @@ -339,7 +332,6 @@ def get_credentials_and_project_id(self) -> tuple[Credentials, str | None]: key_secret_name=key_secret_name, key_secret_project_id=key_secret_project_id, scopes=self.scopes, - delegate_to=self.delegate_to, target_principal=target_principal, delegates=delegates, is_anonymous=is_anonymous, diff --git a/providers/src/airflow/providers/google/common/hooks/discovery_api.py b/providers/src/airflow/providers/google/common/hooks/discovery_api.py index 42e95ef28054..1724e7873a4e 100644 --- a/providers/src/airflow/providers/google/common/hooks/discovery_api.py +++ b/providers/src/airflow/providers/google/common/hooks/discovery_api.py @@ -34,9 +34,6 @@ class GoogleDiscoveryApiHook(GoogleBaseHook): for example 'youtube'. :param api_version: The version of the api that will be requested for example 'v3'. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -54,12 +51,10 @@ def __init__( api_service_name: str, api_version: str, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, ) -> None: super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) self.api_service_name = api_service_name diff --git a/providers/src/airflow/providers/google/marketing_platform/hooks/analytics.py b/providers/src/airflow/providers/google/marketing_platform/hooks/analytics.py deleted file mode 100644 index 0011be9474b5..000000000000 --- a/providers/src/airflow/providers/google/marketing_platform/hooks/analytics.py +++ /dev/null @@ -1,211 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -from typing import Any - -from googleapiclient.discovery import Resource, build -from googleapiclient.http import MediaFileUpload - -from airflow.exceptions import AirflowProviderDeprecationWarning -from airflow.providers.google.common.deprecated import deprecated -from airflow.providers.google.common.hooks.base_google import GoogleBaseHook - - -@deprecated( - planned_removal_date="November 01, 2024", - use_instead="GoogleAnalyticsAdminHook", - reason="The Google Analytics API v3 has sunset and is no longer available as of July 1, 2024.", - category=AirflowProviderDeprecationWarning, -) -class GoogleAnalyticsHook(GoogleBaseHook): - """Hook for Google Analytics 360.""" - - def __init__(self, api_version: str = "v3", *args, **kwargs): - super().__init__(*args, **kwargs) - self.api_version = api_version - self._conn = None - - def _paginate(self, resource: Resource, list_args: dict[str, Any] | None = None) -> list[dict]: - list_args = list_args or {} - result: list[dict] = [] - while True: - # start index has value 1 - request = resource.list(start_index=len(result) + 1, **list_args) - response = request.execute(num_retries=self.num_retries) - result.extend(response.get("items", [])) - # result is the number of fetched links from Analytics - # when all links will be added to the result - # the loop will break - if response["totalResults"] <= len(result): - break - return result - - def get_conn(self) -> Resource: - """Retrieve connection to Google Analytics 360.""" - if not self._conn: - http_authorized = self._authorize() - self._conn = build( - "analytics", - self.api_version, - http=http_authorized, - cache_discovery=False, - ) - return self._conn - - def list_accounts(self) -> list[dict[str, Any]]: - """List accounts list from Google Analytics 360.""" - self.log.info("Retrieving accounts list...") - conn = self.get_conn() - accounts = conn.management().accounts() - result = self._paginate(accounts) - return result - - def get_ad_words_link( - self, account_id: str, web_property_id: str, web_property_ad_words_link_id: str - ) -> dict[str, Any]: - """ - Return a web property-Google Ads link to which the user has access. - - :param account_id: ID of the account which the given web property belongs to. - :param web_property_id: Web property-Google Ads link UA-string. - :param web_property_ad_words_link_id: to retrieve the Google Ads link for. - - :returns: web property-Google Ads - """ - self.log.info("Retrieving ad words links...") - ad_words_link = ( - self.get_conn() - .management() - .webPropertyAdWordsLinks() - .get( - accountId=account_id, - webPropertyId=web_property_id, - webPropertyAdWordsLinkId=web_property_ad_words_link_id, - ) - .execute(num_retries=self.num_retries) - ) - return ad_words_link - - def list_ad_words_links(self, account_id: str, web_property_id: str) -> list[dict[str, Any]]: - """ - List webProperty-Google Ads links for a given web property. - - :param account_id: ID of the account which the given web property belongs to. - :param web_property_id: Web property UA-string to retrieve the Google Ads links for. - - :returns: list of entity Google Ads links. - """ - self.log.info("Retrieving ad words list...") - conn = self.get_conn() - ads_links = conn.management().webPropertyAdWordsLinks() - list_args = {"accountId": account_id, "webPropertyId": web_property_id} - result = self._paginate(ads_links, list_args) - return result - - def upload_data( - self, - file_location: str, - account_id: str, - web_property_id: str, - custom_data_source_id: str, - resumable_upload: bool = False, - ) -> None: - """ - Upload file to GA via the Data Import API. - - :param file_location: The path and name of the file to upload. - :param account_id: The GA account Id to which the data upload belongs. - :param web_property_id: UA-string associated with the upload. - :param custom_data_source_id: Custom Data Source Id to which this data import belongs. - :param resumable_upload: flag to upload the file in a resumable fashion, using a - series of at least two requests. - """ - media = MediaFileUpload( - file_location, - mimetype="application/octet-stream", - resumable=resumable_upload, - ) - - self.log.info( - "Uploading file to GA file for accountId: %s, webPropertyId:%s and customDataSourceId:%s ", - account_id, - web_property_id, - custom_data_source_id, - ) - - self.get_conn().management().uploads().uploadData( - accountId=account_id, - webPropertyId=web_property_id, - customDataSourceId=custom_data_source_id, - media_body=media, - ).execute() - - def delete_upload_data( - self, - account_id: str, - web_property_id: str, - custom_data_source_id: str, - delete_request_body: dict[str, Any], - ) -> None: - """ - Delete the uploaded data for a given account/property/dataset. - - :param account_id: The GA account Id to which the data upload belongs. - :param web_property_id: UA-string associated with the upload. - :param custom_data_source_id: Custom Data Source Id to which this data import belongs. - :param delete_request_body: Dict of customDataImportUids to delete. - """ - self.log.info( - "Deleting previous uploads to GA file for accountId:%s, " - "webPropertyId:%s and customDataSourceId:%s ", - account_id, - web_property_id, - custom_data_source_id, - ) - - self.get_conn().management().uploads().deleteUploadData( - accountId=account_id, - webPropertyId=web_property_id, - customDataSourceId=custom_data_source_id, - body=delete_request_body, - ).execute() - - def list_uploads(self, account_id, web_property_id, custom_data_source_id) -> list[dict[str, Any]]: - """ - Get list of data upload from GA. - - :param account_id: The GA account Id to which the data upload belongs. - :param web_property_id: UA-string associated with the upload. - :param custom_data_source_id: Custom Data Source Id to which this data import belongs. - """ - self.log.info( - "Getting list of uploads for accountId:%s, webPropertyId:%s and customDataSourceId:%s ", - account_id, - web_property_id, - custom_data_source_id, - ) - - uploads = self.get_conn().management().uploads() - list_args = { - "accountId": account_id, - "webPropertyId": web_property_id, - "customDataSourceId": custom_data_source_id, - } - result = self._paginate(uploads, list_args) - return result diff --git a/providers/src/airflow/providers/google/marketing_platform/hooks/campaign_manager.py b/providers/src/airflow/providers/google/marketing_platform/hooks/campaign_manager.py index 08365e41b1c5..4fdc9208c6e8 100644 --- a/providers/src/airflow/providers/google/marketing_platform/hooks/campaign_manager.py +++ b/providers/src/airflow/providers/google/marketing_platform/hooks/campaign_manager.py @@ -39,12 +39,10 @@ def __init__( self, api_version: str = "v4", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, ) -> None: super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) self.api_version = api_version diff --git a/providers/src/airflow/providers/google/marketing_platform/hooks/display_video.py b/providers/src/airflow/providers/google/marketing_platform/hooks/display_video.py index f70f1009c35d..091d29713c47 100644 --- a/providers/src/airflow/providers/google/marketing_platform/hooks/display_video.py +++ b/providers/src/airflow/providers/google/marketing_platform/hooks/display_video.py @@ -35,13 +35,13 @@ def __init__( self, api_version: str = "v2", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, + **kwargs, ) self.api_version = api_version diff --git a/providers/src/airflow/providers/google/marketing_platform/hooks/search_ads.py b/providers/src/airflow/providers/google/marketing_platform/hooks/search_ads.py index d57a89a82182..66f2cdda2f7a 100644 --- a/providers/src/airflow/providers/google/marketing_platform/hooks/search_ads.py +++ b/providers/src/airflow/providers/google/marketing_platform/hooks/search_ads.py @@ -42,10 +42,9 @@ def __init__( self, api_version: str | None = None, gcp_conn_id: str = "google_search_ads_default", + **kwargs, ) -> None: - super().__init__( - gcp_conn_id=gcp_conn_id, - ) + super().__init__(gcp_conn_id=gcp_conn_id, **kwargs) self.api_version = api_version or self.default_api_version def _get_config(self) -> None: @@ -207,13 +206,13 @@ def __init__( self, api_version: str = "v2", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, + **kwargs, ) -> None: super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, + **kwargs, ) self.api_version = api_version diff --git a/providers/src/airflow/providers/google/marketing_platform/operators/analytics.py b/providers/src/airflow/providers/google/marketing_platform/operators/analytics.py deleted file mode 100644 index d35e4c4ab8a2..000000000000 --- a/providers/src/airflow/providers/google/marketing_platform/operators/analytics.py +++ /dev/null @@ -1,551 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -"""This module contains Google Analytics 360 operators.""" - -from __future__ import annotations - -import csv -from tempfile import NamedTemporaryFile -from typing import TYPE_CHECKING, Any, Sequence - -from airflow.exceptions import AirflowProviderDeprecationWarning -from airflow.models import BaseOperator -from airflow.providers.google.cloud.hooks.gcs import GCSHook -from airflow.providers.google.common.deprecated import deprecated -from airflow.providers.google.marketing_platform.hooks.analytics import GoogleAnalyticsHook - -if TYPE_CHECKING: - from airflow.utils.context import Context - - -@deprecated( - planned_removal_date="November 01, 2024", - use_instead="GoogleAnalyticsAdminListAccountsOperator", - reason="The Google Analytics API v3 has sunset and is no longer available as of July 1, 2024.", - category=AirflowProviderDeprecationWarning, -) -class GoogleAnalyticsListAccountsOperator(BaseOperator): - """ - Lists all accounts to which the user has access. - - .. seealso:: - This operator is deprecated, please use - :class:`~airflow.providers.google.marketing_platform.operators.analytics_admin.GoogleAnalyticsAdminListAccountsOperator`: - - .. seealso:: - Check official API docs: - https://developers.google.com/analytics/devguides/config/mgmt/v3/mgmtReference/management/accounts/list - and for python client - http://googleapis.github.io/google-api-python-client/docs/dyn/analytics_v3.management.accounts.html#list - - :param api_version: The version of the api that will be requested for example 'v3'. - :param gcp_conn_id: The connection ID to use when fetching connection info. - :param impersonation_chain: Optional service account to impersonate using short-term - credentials, or chained list of accounts required to get the access_token - of the last account in the list, which will be impersonated in the request. - If set as a string, the account must grant the originating account - the Service Account Token Creator IAM role. - If set as a sequence, the identities from the list must grant - Service Account Token Creator IAM role to the directly preceding identity, with first - account from the list granting this role to the originating account (templated). - """ - - template_fields: Sequence[str] = ( - "api_version", - "gcp_conn_id", - "impersonation_chain", - ) - - def __init__( - self, - *, - api_version: str = "v3", - gcp_conn_id: str = "google_cloud_default", - impersonation_chain: str | Sequence[str] | None = None, - **kwargs, - ) -> None: - super().__init__(**kwargs) - - self.api_version = api_version - self.gcp_conn_id = gcp_conn_id - self.impersonation_chain = impersonation_chain - - def execute(self, context: Context) -> list[dict[str, Any]]: - hook = GoogleAnalyticsHook( - api_version=self.api_version, - gcp_conn_id=self.gcp_conn_id, - impersonation_chain=self.impersonation_chain, - ) - result = hook.list_accounts() - return result - - -@deprecated( - planned_removal_date="November 01, 2024", - use_instead="GoogleAnalyticsAdminGetGoogleAdsLinkOperator", - reason="The Google Analytics API v3 has sunset and is no longer available as of July 1, 2024.", - category=AirflowProviderDeprecationWarning, -) -class GoogleAnalyticsGetAdsLinkOperator(BaseOperator): - """ - Returns a web property-Google Ads link to which the user has access. - - .. seealso:: - This operator is deprecated, please use - :class:`~airflow.providers.google.marketing_platform.operators.analytics_admin.GoogleAnalyticsAdminGetGoogleAdsLinkOperator`: - - .. seealso:: - Check official API docs: - https://developers.google.com/analytics/devguides/config/mgmt/v3/mgmtReference/management/webPropertyAdWordsLinks/get - - - :param account_id: ID of the account which the given web property belongs to. - :param web_property_ad_words_link_id: Web property-Google Ads link ID. - :param web_property_id: Web property ID to retrieve the Google Ads link for. - :param impersonation_chain: Optional service account to impersonate using short-term - credentials, or chained list of accounts required to get the access_token - of the last account in the list, which will be impersonated in the request. - If set as a string, the account must grant the originating account - the Service Account Token Creator IAM role. - If set as a sequence, the identities from the list must grant - Service Account Token Creator IAM role to the directly preceding identity, with first - account from the list granting this role to the originating account (templated). - """ - - template_fields: Sequence[str] = ( - "api_version", - "gcp_conn_id", - "account_id", - "web_property_ad_words_link_id", - "web_property_id", - "impersonation_chain", - ) - - def __init__( - self, - *, - account_id: str, - web_property_ad_words_link_id: str, - web_property_id: str, - api_version: str = "v3", - gcp_conn_id: str = "google_cloud_default", - impersonation_chain: str | Sequence[str] | None = None, - **kwargs, - ): - super().__init__(**kwargs) - - self.account_id = account_id - self.web_property_ad_words_link_id = web_property_ad_words_link_id - self.web_property_id = web_property_id - self.api_version = api_version - self.gcp_conn_id = gcp_conn_id - self.impersonation_chain = impersonation_chain - - def execute(self, context: Context) -> dict[str, Any]: - hook = GoogleAnalyticsHook( - api_version=self.api_version, - gcp_conn_id=self.gcp_conn_id, - impersonation_chain=self.impersonation_chain, - ) - result = hook.get_ad_words_link( - account_id=self.account_id, - web_property_id=self.web_property_id, - web_property_ad_words_link_id=self.web_property_ad_words_link_id, - ) - return result - - -@deprecated( - planned_removal_date="November 01, 2024", - use_instead="GoogleAnalyticsAdminListGoogleAdsLinksOperator", - reason="The Google Analytics API v3 has sunset and is no longer available as of July 1, 2024.", - category=AirflowProviderDeprecationWarning, -) -class GoogleAnalyticsRetrieveAdsLinksListOperator(BaseOperator): - """ - Lists webProperty-Google Ads links for a given web property. - - .. seealso:: - This operator is deprecated, please use - :class:`~airflow.providers.google.marketing_platform.operators.analytics_admin.GoogleAnalyticsAdminListGoogleAdsLinksOperator`: - - .. seealso:: - Check official API docs: - https://developers.google.com/analytics/devguides/config/mgmt/v3/mgmtReference/management/webPropertyAdWordsLinks/list#http-request - - :param account_id: ID of the account which the given web property belongs to. - :param web_property_id: Web property UA-string to retrieve the Google Ads links for. - :param impersonation_chain: Optional service account to impersonate using short-term - credentials, or chained list of accounts required to get the access_token - of the last account in the list, which will be impersonated in the request. - If set as a string, the account must grant the originating account - the Service Account Token Creator IAM role. - If set as a sequence, the identities from the list must grant - Service Account Token Creator IAM role to the directly preceding identity, with first - account from the list granting this role to the originating account (templated). - """ - - template_fields: Sequence[str] = ( - "api_version", - "gcp_conn_id", - "account_id", - "web_property_id", - "impersonation_chain", - ) - - def __init__( - self, - *, - account_id: str, - web_property_id: str, - api_version: str = "v3", - gcp_conn_id: str = "google_cloud_default", - impersonation_chain: str | Sequence[str] | None = None, - **kwargs, - ) -> None: - super().__init__(**kwargs) - - self.account_id = account_id - self.web_property_id = web_property_id - self.api_version = api_version - self.gcp_conn_id = gcp_conn_id - self.impersonation_chain = impersonation_chain - - def execute(self, context: Context) -> list[dict[str, Any]]: - hook = GoogleAnalyticsHook( - api_version=self.api_version, - gcp_conn_id=self.gcp_conn_id, - impersonation_chain=self.impersonation_chain, - ) - result = hook.list_ad_words_links( - account_id=self.account_id, - web_property_id=self.web_property_id, - ) - return result - - -@deprecated( - planned_removal_date="November 01, 2024", - use_instead="GoogleAnalyticsAdminCreateDataStreamOperator", - reason="The Google Analytics API v3 has sunset and is no longer available as of July 1, 2024.", - category=AirflowProviderDeprecationWarning, -) -class GoogleAnalyticsDataImportUploadOperator(BaseOperator): - """ - Take a file from Cloud Storage and uploads it to GA via data import API. - - .. seealso:: - This operator is deprecated, please use - :class:`~airflow.providers.google.marketing_platform.operators.analytics_admin.GoogleAnalyticsAdminCreateDataStreamOperator`: - - :param storage_bucket: The Google cloud storage bucket where the file is stored. - :param storage_name_object: The name of the object in the desired Google cloud - storage bucket. (templated) If the destination points to an existing - folder, the file will be taken from the specified folder. - :param account_id: The GA account Id (long) to which the data upload belongs. - :param web_property_id: The web property UA-string associated with the upload. - :param custom_data_source_id: The id to which the data import belongs - :param resumable_upload: flag to upload the file in a resumable fashion, using a - series of at least two requests. - :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. - :param api_version: The version of the api that will be requested for example 'v3'. - :param impersonation_chain: Optional service account to impersonate using short-term - credentials, or chained list of accounts required to get the access_token - of the last account in the list, which will be impersonated in the request. - If set as a string, the account must grant the originating account - the Service Account Token Creator IAM role. - If set as a sequence, the identities from the list must grant - Service Account Token Creator IAM role to the directly preceding identity, with first - account from the list granting this role to the originating account (templated). - """ - - template_fields: Sequence[str] = ( - "storage_bucket", - "storage_name_object", - "impersonation_chain", - ) - - def __init__( - self, - *, - storage_bucket: str, - storage_name_object: str, - account_id: str, - web_property_id: str, - custom_data_source_id: str, - resumable_upload: bool = False, - gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, - api_version: str = "v3", - impersonation_chain: str | Sequence[str] | None = None, - **kwargs, - ) -> None: - super().__init__(**kwargs) - self.storage_bucket = storage_bucket - self.storage_name_object = storage_name_object - self.account_id = account_id - self.web_property_id = web_property_id - self.custom_data_source_id = custom_data_source_id - self.resumable_upload = resumable_upload - self.gcp_conn_id = gcp_conn_id - self.delegate_to = delegate_to - self.api_version = api_version - self.impersonation_chain = impersonation_chain - - def execute(self, context: Context) -> None: - gcs_hook = GCSHook( - gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, - impersonation_chain=self.impersonation_chain, - ) - - ga_hook = GoogleAnalyticsHook( - gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, - api_version=self.api_version, - impersonation_chain=self.impersonation_chain, - ) - - with NamedTemporaryFile("w+") as tmp_file: - self.log.info( - "Downloading file from GCS: %s/%s ", - self.storage_bucket, - self.storage_name_object, - ) - gcs_hook.download( - bucket_name=self.storage_bucket, - object_name=self.storage_name_object, - filename=tmp_file.name, - ) - - ga_hook.upload_data( - tmp_file.name, - self.account_id, - self.web_property_id, - self.custom_data_source_id, - self.resumable_upload, - ) - - -@deprecated( - planned_removal_date="November 01, 2024", - use_instead="GoogleAnalyticsAdminDeleteDataStreamOperator", - reason="The Google Analytics API v3 has sunset and is no longer available as of July 1, 2024.", - category=AirflowProviderDeprecationWarning, -) -class GoogleAnalyticsDeletePreviousDataUploadsOperator(BaseOperator): - """ - Deletes previous GA uploads to leave the latest file to control the size of the Data Set Quota. - - .. seealso:: - This operator is deprecated, please use - :class:`~airflow.providers.google.marketing_platform.operators.analytics_admin.GoogleAnalyticsAdminDeleteDataStreamOperator`: - - :param account_id: The GA account Id (long) to which the data upload belongs. - :param web_property_id: The web property UA-string associated with the upload. - :param custom_data_source_id: The id to which the data import belongs. - :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. - :param api_version: The version of the api that will be requested for example 'v3'. - :param impersonation_chain: Optional service account to impersonate using short-term - credentials, or chained list of accounts required to get the access_token - of the last account in the list, which will be impersonated in the request. - If set as a string, the account must grant the originating account - the Service Account Token Creator IAM role. - If set as a sequence, the identities from the list must grant - Service Account Token Creator IAM role to the directly preceding identity, with first - account from the list granting this role to the originating account (templated). - """ - - template_fields: Sequence[str] = ("impersonation_chain",) - - def __init__( - self, - account_id: str, - web_property_id: str, - custom_data_source_id: str, - gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, - api_version: str = "v3", - impersonation_chain: str | Sequence[str] | None = None, - **kwargs, - ) -> None: - super().__init__(**kwargs) - - self.account_id = account_id - self.web_property_id = web_property_id - self.custom_data_source_id = custom_data_source_id - self.gcp_conn_id = gcp_conn_id - self.delegate_to = delegate_to - self.api_version = api_version - self.impersonation_chain = impersonation_chain - - def execute(self, context: Context) -> None: - ga_hook = GoogleAnalyticsHook( - gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, - api_version=self.api_version, - impersonation_chain=self.impersonation_chain, - ) - - uploads = ga_hook.list_uploads( - account_id=self.account_id, - web_property_id=self.web_property_id, - custom_data_source_id=self.custom_data_source_id, - ) - - cids = [upload["id"] for upload in uploads] - delete_request_body = {"customDataImportUids": cids} - - ga_hook.delete_upload_data( - self.account_id, - self.web_property_id, - self.custom_data_source_id, - delete_request_body, - ) - - -class GoogleAnalyticsModifyFileHeadersDataImportOperator(BaseOperator): - """ - GA has a very particular naming convention for Data Import. - - Ability to prefix "ga:" to all column headers and also a dict to rename columns to - match the custom dimension ID in GA i.e clientId : dimensionX. - - :param storage_bucket: The Google cloud storage bucket where the file is stored. - :param storage_name_object: The name of the object in the desired Google cloud - storage bucket. (templated) If the destination points to an existing - folder, the file will be taken from the specified folder. - :param gcp_conn_id: The connection ID to use when fetching connection info. - :param custom_dimension_header_mapping: Dictionary to handle when uploading - custom dimensions which have generic IDs ie. 'dimensionX' which are - set by GA. Dictionary maps the current CSV header to GA ID which will - be the new header for the CSV to upload to GA eg clientId : dimension1. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. - :param impersonation_chain: Optional service account to impersonate using short-term - credentials, or chained list of accounts required to get the access_token - of the last account in the list, which will be impersonated in the request. - If set as a string, the account must grant the originating account - the Service Account Token Creator IAM role. - If set as a sequence, the identities from the list must grant - Service Account Token Creator IAM role to the directly preceding identity, with first - account from the list granting this role to the originating account (templated). - """ - - template_fields: Sequence[str] = ( - "storage_bucket", - "storage_name_object", - "impersonation_chain", - ) - - def __init__( - self, - storage_bucket: str, - storage_name_object: str, - gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, - custom_dimension_header_mapping: dict[str, str] | None = None, - impersonation_chain: str | Sequence[str] | None = None, - **kwargs, - ) -> None: - super().__init__(**kwargs) - self.storage_bucket = storage_bucket - self.storage_name_object = storage_name_object - self.gcp_conn_id = gcp_conn_id - self.delegate_to = delegate_to - self.custom_dimension_header_mapping = custom_dimension_header_mapping or {} - self.impersonation_chain = impersonation_chain - - def _modify_column_headers( - self, tmp_file_location: str, custom_dimension_header_mapping: dict[str, str] - ) -> None: - # Check headers - self.log.info("Checking if file contains headers") - with open(tmp_file_location) as check_header_file: - has_header = csv.Sniffer().has_header(check_header_file.read(1024)) - if not has_header: - raise NameError( - "CSV does not contain headers, please add them " - "to use the modify column headers functionality" - ) - - # Transform - self.log.info("Modifying column headers to be compatible for data upload") - with open(tmp_file_location) as read_file: - reader = csv.reader(read_file) - headers = next(reader) - new_headers = [] - for header in headers: - if header in custom_dimension_header_mapping: - header = custom_dimension_header_mapping.get(header) # type: ignore - new_header = f"ga:{header}" - new_headers.append(new_header) - all_data = read_file.readlines() - final_headers = ",".join(new_headers) + "\n" - all_data.insert(0, final_headers) - - # Save result - self.log.info("Saving transformed file") - with open(tmp_file_location, "w") as write_file: - write_file.writelines(all_data) - - def execute(self, context: Context) -> None: - gcs_hook = GCSHook( - gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, - impersonation_chain=self.impersonation_chain, - ) - with NamedTemporaryFile("w+") as tmp_file: - # Download file from GCS - self.log.info( - "Downloading file from GCS: %s/%s ", - self.storage_bucket, - self.storage_name_object, - ) - - gcs_hook.download( - bucket_name=self.storage_bucket, - object_name=self.storage_name_object, - filename=tmp_file.name, - ) - - # Modify file - self.log.info("Modifying temporary file %s", tmp_file.name) - self._modify_column_headers( - tmp_file_location=tmp_file.name, - custom_dimension_header_mapping=self.custom_dimension_header_mapping, - ) - - # Upload newly formatted file to cloud storage - self.log.info( - "Uploading file to GCS: %s/%s ", - self.storage_bucket, - self.storage_name_object, - ) - gcs_hook.upload( - bucket_name=self.storage_bucket, - object_name=self.storage_name_object, - filename=tmp_file.name, - ) diff --git a/providers/src/airflow/providers/google/marketing_platform/operators/campaign_manager.py b/providers/src/airflow/providers/google/marketing_platform/operators/campaign_manager.py index 918aa389dff7..1a286f02037b 100644 --- a/providers/src/airflow/providers/google/marketing_platform/operators/campaign_manager.py +++ b/providers/src/airflow/providers/google/marketing_platform/operators/campaign_manager.py @@ -52,9 +52,6 @@ class GoogleCampaignManagerDeleteReportOperator(BaseOperator): :param report_id: The ID of the report. :param api_version: The version of the api that will be requested, for example 'v4'. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -71,7 +68,6 @@ class GoogleCampaignManagerDeleteReportOperator(BaseOperator): "report_name", "api_version", "gcp_conn_id", - "delegate_to", "impersonation_chain", ) @@ -83,7 +79,6 @@ def __init__( report_id: str | None = None, api_version: str = "v4", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -98,13 +93,11 @@ def __init__( self.report_id = report_id self.api_version = api_version self.gcp_conn_id = gcp_conn_id - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context) -> None: hook = GoogleCampaignManagerHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, api_version=self.api_version, impersonation_chain=self.impersonation_chain, ) @@ -143,9 +136,6 @@ class GoogleCampaignManagerDownloadReportOperator(BaseOperator): :param chunk_size: File will be downloaded in chunks of this many bytes. :param api_version: The version of the api that will be requested, for example 'v4'. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -165,7 +155,6 @@ class GoogleCampaignManagerDownloadReportOperator(BaseOperator): "chunk_size", "api_version", "gcp_conn_id", - "delegate_to", "impersonation_chain", ) @@ -181,7 +170,6 @@ def __init__( chunk_size: int = 10 * 1024 * 1024, api_version: str = "v4", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -195,7 +183,6 @@ def __init__( self.bucket_name = bucket_name self.report_name = report_name self.gcp_conn_id = gcp_conn_id - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def _resolve_file_name(self, name: str) -> str: @@ -215,13 +202,11 @@ def _set_bucket_name(name: str) -> str: def execute(self, context: Context) -> None: hook = GoogleCampaignManagerHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, api_version=self.api_version, impersonation_chain=self.impersonation_chain, ) gcs_hook = GCSHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) # Get name of the report @@ -270,9 +255,6 @@ class GoogleCampaignManagerInsertReportOperator(BaseOperator): :param report: Report to be created. :param api_version: The version of the api that will be requested, for example 'v4'. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -288,7 +270,6 @@ class GoogleCampaignManagerInsertReportOperator(BaseOperator): "report", "api_version", "gcp_conn_id", - "delegate_to", "impersonation_chain", ) @@ -301,7 +282,6 @@ def __init__( report: dict[str, Any], api_version: str = "v4", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -310,7 +290,6 @@ def __init__( self.report = report self.api_version = api_version self.gcp_conn_id = gcp_conn_id - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def prepare_template(self) -> None: @@ -322,7 +301,6 @@ def prepare_template(self) -> None: def execute(self, context: Context): hook = GoogleCampaignManagerHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, api_version=self.api_version, impersonation_chain=self.impersonation_chain, ) @@ -351,9 +329,6 @@ class GoogleCampaignManagerRunReportOperator(BaseOperator): :param synchronous: If set and true, tries to run the report synchronously. :param api_version: The version of the api that will be requested, for example 'v4'. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -370,7 +345,6 @@ class GoogleCampaignManagerRunReportOperator(BaseOperator): "synchronous", "api_version", "gcp_conn_id", - "delegate_to", "impersonation_chain", ) @@ -382,7 +356,6 @@ def __init__( synchronous: bool = False, api_version: str = "v4", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -392,13 +365,11 @@ def __init__( self.synchronous = synchronous self.api_version = api_version self.gcp_conn_id = gcp_conn_id - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context): hook = GoogleCampaignManagerHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, api_version=self.api_version, impersonation_chain=self.impersonation_chain, ) @@ -438,9 +409,6 @@ class GoogleCampaignManagerBatchInsertConversionsOperator(BaseOperator): :param max_failed_inserts: The maximum number of conversions that failed to be inserted :param api_version: The version of the api that will be requested, for example 'v4'. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -471,7 +439,6 @@ def __init__( max_failed_inserts: int = 0, api_version: str = "v4", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -484,13 +451,11 @@ def __init__( self.max_failed_inserts = max_failed_inserts self.api_version = api_version self.gcp_conn_id = gcp_conn_id - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context): hook = GoogleCampaignManagerHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, api_version=self.api_version, impersonation_chain=self.impersonation_chain, ) @@ -529,9 +494,6 @@ class GoogleCampaignManagerBatchUpdateConversionsOperator(BaseOperator): :param max_failed_updates: The maximum number of conversions that failed to be updated :param api_version: The version of the api that will be requested, for example 'v4'. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -562,7 +524,6 @@ def __init__( max_failed_updates: int = 0, api_version: str = "v4", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -575,13 +536,11 @@ def __init__( self.max_failed_updates = max_failed_updates self.api_version = api_version self.gcp_conn_id = gcp_conn_id - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context): hook = GoogleCampaignManagerHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, api_version=self.api_version, impersonation_chain=self.impersonation_chain, ) diff --git a/providers/src/airflow/providers/google/marketing_platform/operators/display_video.py b/providers/src/airflow/providers/google/marketing_platform/operators/display_video.py index 015f3f2db265..8dadff754062 100644 --- a/providers/src/airflow/providers/google/marketing_platform/operators/display_video.py +++ b/providers/src/airflow/providers/google/marketing_platform/operators/display_video.py @@ -52,9 +52,6 @@ class GoogleDisplayVideo360CreateQueryOperator(BaseOperator): https://developers.google.com/bid-manager/v2/queries#Query :param api_version: The version of the api that will be requested for example 'v3'. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -77,7 +74,6 @@ def __init__( body: dict[str, Any], api_version: str = "v2", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -85,7 +81,6 @@ def __init__( self.body = body self.api_version = api_version self.gcp_conn_id = gcp_conn_id - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def prepare_template(self) -> None: @@ -97,7 +92,6 @@ def prepare_template(self) -> None: def execute(self, context: Context) -> dict: hook = GoogleDisplayVideo360Hook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, api_version=self.api_version, impersonation_chain=self.impersonation_chain, ) @@ -125,9 +119,6 @@ class GoogleDisplayVideo360DeleteReportOperator(BaseOperator): :param report_name: Name of the report to delete. :param api_version: The version of the api that will be requested for example 'v3'. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -150,7 +141,6 @@ def __init__( report_name: str | None = None, api_version: str = "v2", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -159,7 +149,6 @@ def __init__( self.report_name = report_name self.api_version = api_version self.gcp_conn_id = gcp_conn_id - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain if report_name and report_id: @@ -171,7 +160,6 @@ def __init__( def execute(self, context: Context) -> None: hook = GoogleDisplayVideo360Hook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, api_version=self.api_version, impersonation_chain=self.impersonation_chain, ) @@ -208,9 +196,6 @@ class GoogleDisplayVideo360DownloadReportV2Operator(BaseOperator): :param gzip: Option to compress local file or file data for upload :param api_version: The version of the api that will be requested for example 'v3'. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -240,7 +225,6 @@ def __init__( chunk_size: int = 10 * 1024 * 1024, api_version: str = "v2", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -253,7 +237,6 @@ def __init__( self.report_name = report_name self.api_version = api_version self.gcp_conn_id = gcp_conn_id - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def _resolve_file_name(self, name: str) -> str: @@ -269,13 +252,11 @@ def _set_bucket_name(name: str) -> str: def execute(self, context: Context): hook = GoogleDisplayVideo360Hook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, api_version=self.api_version, impersonation_chain=self.impersonation_chain, ) gcs_hook = GCSHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) @@ -333,9 +314,6 @@ class GoogleDisplayVideo360RunQueryOperator(BaseOperator): https://developers.google.com/bid-manager/v2/queries/run :param api_version: The version of the api that will be requested for example 'v3'. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -359,7 +337,6 @@ def __init__( parameters: dict[str, Any] | None = None, api_version: str = "v2", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -367,14 +344,12 @@ def __init__( self.query_id = query_id self.api_version = api_version self.gcp_conn_id = gcp_conn_id - self.delegate_to = delegate_to self.parameters = parameters self.impersonation_chain = impersonation_chain def execute(self, context: Context) -> dict: hook = GoogleDisplayVideo360Hook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, api_version=self.api_version, impersonation_chain=self.impersonation_chain, ) @@ -422,7 +397,6 @@ def __init__( gzip: bool = False, api_version: str = "v1.1", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -433,19 +407,16 @@ def __init__( self.gzip = gzip self.api_version = api_version self.gcp_conn_id = gcp_conn_id - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context) -> str: gcs_hook = GCSHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) hook = GoogleDisplayVideo360Hook( gcp_conn_id=self.gcp_conn_id, api_version=self.api_version, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) @@ -498,7 +469,6 @@ def __init__( object_name: str, api_version: str = "v1.1", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -507,18 +477,15 @@ def __init__( self.object_name = object_name self.api_version = api_version self.gcp_conn_id = gcp_conn_id - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context) -> None: gcs_hook = GCSHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) hook = GoogleDisplayVideo360Hook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, api_version=self.api_version, impersonation_chain=self.impersonation_chain, ) @@ -555,9 +522,6 @@ class GoogleDisplayVideo360CreateSDFDownloadTaskOperator(BaseOperator): :param id_filter: Filters on entities by their entity IDs. :param inventory_source_filter: Filters on Inventory Sources by their IDs. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -579,7 +543,6 @@ def __init__( body_request: dict[str, Any], api_version: str = "v1", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -587,13 +550,11 @@ def __init__( self.body_request = body_request self.api_version = api_version self.gcp_conn_id = gcp_conn_id - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context) -> dict[str, Any]: hook = GoogleDisplayVideo360Hook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, api_version=self.api_version, impersonation_chain=self.impersonation_chain, ) @@ -627,9 +588,6 @@ class GoogleDisplayVideo360SDFtoGCSOperator(BaseOperator): :param id_filter: Filters on entities by their entity IDs. :param inventory_source_filter: Filters on Inventory Sources by their IDs. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -656,7 +614,6 @@ def __init__( gzip: bool = False, api_version: str = "v1", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -667,19 +624,16 @@ def __init__( self.gzip = gzip self.api_version = api_version self.gcp_conn_id = gcp_conn_id - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Context) -> str: hook = GoogleDisplayVideo360Hook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, api_version=self.api_version, impersonation_chain=self.impersonation_chain, ) gcs_hook = GCSHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) diff --git a/providers/src/airflow/providers/google/marketing_platform/sensors/campaign_manager.py b/providers/src/airflow/providers/google/marketing_platform/sensors/campaign_manager.py index 446426280a64..316e7bfa59f4 100644 --- a/providers/src/airflow/providers/google/marketing_platform/sensors/campaign_manager.py +++ b/providers/src/airflow/providers/google/marketing_platform/sensors/campaign_manager.py @@ -45,9 +45,6 @@ class GoogleCampaignManagerReportSensor(BaseSensorOperator): :param file_id: The ID of the report file. :param api_version: The version of the api that will be requested, for example 'v4'. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -68,7 +65,6 @@ class GoogleCampaignManagerReportSensor(BaseSensorOperator): def poke(self, context: Context) -> bool: hook = GoogleCampaignManagerHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, api_version=self.api_version, impersonation_chain=self.impersonation_chain, ) @@ -84,7 +80,6 @@ def __init__( file_id: str, api_version: str = "v4", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, mode: str = "reschedule", poke_interval: int = 60 * 5, impersonation_chain: str | Sequence[str] | None = None, @@ -98,5 +93,4 @@ def __init__( self.file_id = file_id self.api_version = api_version self.gcp_conn_id = gcp_conn_id - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain diff --git a/providers/src/airflow/providers/google/marketing_platform/sensors/display_video.py b/providers/src/airflow/providers/google/marketing_platform/sensors/display_video.py index 869317afa46e..5a5a143d3946 100644 --- a/providers/src/airflow/providers/google/marketing_platform/sensors/display_video.py +++ b/providers/src/airflow/providers/google/marketing_platform/sensors/display_video.py @@ -39,9 +39,6 @@ class GoogleDisplayVideo360GetSDFDownloadOperationSensor(BaseSensorOperator): :param operation_name: The name of the operation resource :param api_version: The version of the api that will be requested for example 'v1'. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -63,7 +60,6 @@ def __init__( operation_name: str, api_version: str = "v1", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, mode: str = "reschedule", poke_interval: int = 60 * 5, impersonation_chain: str | Sequence[str] | None = None, @@ -76,13 +72,11 @@ def __init__( self.operation_name = operation_name self.api_version = api_version self.gcp_conn_id = gcp_conn_id - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def poke(self, context: Context) -> bool: hook = GoogleDisplayVideo360Hook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, api_version=self.api_version, impersonation_chain=self.impersonation_chain, ) @@ -107,9 +101,6 @@ class GoogleDisplayVideo360RunQuerySensor(BaseSensorOperator): :param report_id: Report ID for which you want to wait :param api_version: The version of the api that will be requested for example 'v3'. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -133,7 +124,6 @@ def __init__( report_id: str, api_version: str = "v2", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -142,13 +132,11 @@ def __init__( self.report_id = report_id self.api_version = api_version self.gcp_conn_id = gcp_conn_id - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def poke(self, context: Context) -> bool: hook = GoogleDisplayVideo360Hook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, api_version=self.api_version, impersonation_chain=self.impersonation_chain, ) diff --git a/providers/src/airflow/providers/google/provider.yaml b/providers/src/airflow/providers/google/provider.yaml index 604eff4190fc..6c3350a7ba4f 100644 --- a/providers/src/airflow/providers/google/provider.yaml +++ b/providers/src/airflow/providers/google/provider.yaml @@ -208,12 +208,6 @@ integrations: how-to-guide: - /docs/apache-airflow-providers-google/operators/marketing_platform/analytics_admin.rst tags: [gmp] - - integration-name: Google Analytics360 - external-doc-url: https://analytics.google.com/ - logo: /integration-logos/gcp/Google-Analytics.png - how-to-guide: - - /docs/apache-airflow-providers-google/operators/marketing_platform/analytics.rst - tags: [gmp] - integration-name: Google Ads external-doc-url: https://ads.google.com/ logo: /integration-logos/gcp/Google-Ads.png @@ -671,9 +665,6 @@ operators: - integration-name: Google Analytics (GA4) python-modules: - airflow.providers.google.marketing_platform.operators.analytics_admin - - integration-name: Google Analytics360 - python-modules: - - airflow.providers.google.marketing_platform.operators.analytics - integration-name: Google Campaign Manager python-modules: - airflow.providers.google.marketing_platform.operators.campaign_manager @@ -937,9 +928,6 @@ hooks: - integration-name: Google Analytics (GA4) python-modules: - airflow.providers.google.marketing_platform.hooks.analytics_admin - - integration-name: Google Analytics360 - python-modules: - - airflow.providers.google.marketing_platform.hooks.analytics - integration-name: Google Campaign Manager python-modules: - airflow.providers.google.marketing_platform.hooks.campaign_manager @@ -1190,8 +1178,6 @@ connection-types: connection-type: leveldb extra-links: - - airflow.providers.google.cloud.operators.bigquery.BigQueryConsoleLink - - airflow.providers.google.cloud.operators.bigquery.BigQueryConsoleIndexableLink - airflow.providers.google.cloud.links.dataform.DataformRepositoryLink - airflow.providers.google.cloud.links.dataform.DataformWorkspaceLink - airflow.providers.google.cloud.links.dataform.DataformWorkflowInvocationLink diff --git a/providers/src/airflow/providers/google/suite/hooks/calendar.py b/providers/src/airflow/providers/google/suite/hooks/calendar.py index 09697233cbb9..b64ede9dc9a6 100644 --- a/providers/src/airflow/providers/google/suite/hooks/calendar.py +++ b/providers/src/airflow/providers/google/suite/hooks/calendar.py @@ -38,9 +38,6 @@ class GoogleCalendarHook(GoogleBaseHook): :param gcp_conn_id: The connection ID to use when fetching connection info. :param api_version: API Version. For example v3 - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -55,17 +52,13 @@ def __init__( self, api_version: str, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, ) -> None: super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) - self.gcp_conn_id = gcp_conn_id self.api_version = api_version - self.delegate_to = delegate_to self._conn = None def get_conn(self) -> Any: diff --git a/providers/src/airflow/providers/google/suite/hooks/drive.py b/providers/src/airflow/providers/google/suite/hooks/drive.py index 8c7884dc08b9..62f01499d845 100644 --- a/providers/src/airflow/providers/google/suite/hooks/drive.py +++ b/providers/src/airflow/providers/google/suite/hooks/drive.py @@ -34,9 +34,6 @@ class GoogleDriveHook(GoogleBaseHook): :param api_version: API version used (for example v3). :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -53,12 +50,10 @@ def __init__( self, api_version: str = "v3", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, ) -> None: super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) self.api_version = api_version diff --git a/providers/src/airflow/providers/google/suite/hooks/sheets.py b/providers/src/airflow/providers/google/suite/hooks/sheets.py index 7b8de0be6538..992cb9e6ac55 100644 --- a/providers/src/airflow/providers/google/suite/hooks/sheets.py +++ b/providers/src/airflow/providers/google/suite/hooks/sheets.py @@ -35,9 +35,6 @@ class GSheetsHook(GoogleBaseHook): :param gcp_conn_id: The connection ID to use when fetching connection info. :param api_version: API Version - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -52,17 +49,14 @@ def __init__( self, gcp_conn_id: str = "google_cloud_default", api_version: str = "v4", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, ) -> None: super().__init__( gcp_conn_id=gcp_conn_id, - delegate_to=delegate_to, impersonation_chain=impersonation_chain, ) self.gcp_conn_id = gcp_conn_id self.api_version = api_version - self.delegate_to = delegate_to self._conn = None def get_conn(self) -> Any: diff --git a/providers/src/airflow/providers/google/suite/operators/sheets.py b/providers/src/airflow/providers/google/suite/operators/sheets.py index 71bdae50cc6f..4f3e581b55ed 100644 --- a/providers/src/airflow/providers/google/suite/operators/sheets.py +++ b/providers/src/airflow/providers/google/suite/operators/sheets.py @@ -33,9 +33,6 @@ class GoogleSheetsCreateSpreadsheetOperator(BaseOperator): :param spreadsheet: an instance of Spreadsheet https://developers.google.com/sheets/api/reference/rest/v4/spreadsheets#Spreadsheet :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -56,20 +53,17 @@ def __init__( *, spreadsheet: dict[str, Any], gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: super().__init__(**kwargs) self.gcp_conn_id = gcp_conn_id self.spreadsheet = spreadsheet - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def execute(self, context: Any) -> dict[str, Any]: hook = GSheetsHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) spreadsheet = hook.create_spreadsheet(spreadsheet=self.spreadsheet) diff --git a/providers/src/airflow/providers/google/suite/sensors/drive.py b/providers/src/airflow/providers/google/suite/sensors/drive.py index 3c223f3df8fa..b66a2059b2d4 100644 --- a/providers/src/airflow/providers/google/suite/sensors/drive.py +++ b/providers/src/airflow/providers/google/suite/sensors/drive.py @@ -37,9 +37,6 @@ class GoogleDriveFileExistenceSensor(BaseSensorOperator): :param drive_id: Optional. The id of the shared Google Drive in which the file resides. :param gcp_conn_id: The connection ID to use when connecting to Google Cloud Storage. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -65,7 +62,6 @@ def __init__( file_name: str, drive_id: str | None = None, gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -74,14 +70,12 @@ def __init__( self.file_name = file_name self.drive_id = drive_id self.gcp_conn_id = gcp_conn_id - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def poke(self, context: Context) -> bool: self.log.info("Sensor is checking for the file %s in the folder %s", self.file_name, self.folder_id) hook = GoogleDriveHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) return hook.exists(folder_id=self.folder_id, file_name=self.file_name, drive_id=self.drive_id) diff --git a/providers/src/airflow/providers/google/suite/transfers/gcs_to_gdrive.py b/providers/src/airflow/providers/google/suite/transfers/gcs_to_gdrive.py index 39b58bc1424b..c290b349fbd8 100644 --- a/providers/src/airflow/providers/google/suite/transfers/gcs_to_gdrive.py +++ b/providers/src/airflow/providers/google/suite/transfers/gcs_to_gdrive.py @@ -80,9 +80,6 @@ class GCSToGoogleDriveOperator(BaseOperator): If set as a sequence, the identities from the list must grant Service Account Token Creator IAM role to the directly preceding identity, with first account from the list granting this role to the originating account (templated). - :param delegate_to: (Optional) The account to impersonate using domain-wide delegation - of authority, if any. For this to work, the service account making the - request must have domain-wide delegation enabled. This only applies to the Google Drive connection. """ template_fields: Sequence[str] = ( @@ -103,7 +100,6 @@ def __init__( move_object: bool = False, gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, - delegate_to: str | None = None, **kwargs, ) -> None: super().__init__(**kwargs) @@ -115,7 +111,6 @@ def __init__( self.move_object = move_object self.gcp_conn_id = gcp_conn_id self.impersonation_chain = impersonation_chain - self.delegate_to = delegate_to self.gcs_hook: GCSHook | None = None self.gdrive_hook: GoogleDriveHook | None = None @@ -127,7 +122,6 @@ def execute(self, context: Context): self.gdrive_hook = GoogleDriveHook( gcp_conn_id=self.gcp_conn_id, impersonation_chain=self.impersonation_chain, - delegate_to=self.delegate_to, ) if WILDCARD in self.source_object: diff --git a/providers/src/airflow/providers/google/suite/transfers/gcs_to_sheets.py b/providers/src/airflow/providers/google/suite/transfers/gcs_to_sheets.py index 0a49ad414416..a2e6d7ca45db 100644 --- a/providers/src/airflow/providers/google/suite/transfers/gcs_to_sheets.py +++ b/providers/src/airflow/providers/google/suite/transfers/gcs_to_sheets.py @@ -38,9 +38,6 @@ class GCSToGoogleSheetsOperator(BaseOperator): :param object_name: Path to the .csv file on the GCS bucket. :param spreadsheet_range: The A1 notation of the values to retrieve. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. This only applies to the Google Sheet Connection :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -68,7 +65,6 @@ def __init__( spreadsheet_range: str = "Sheet1", gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, - delegate_to: str | None = None, **kwargs, ) -> None: super().__init__(**kwargs) @@ -79,13 +75,11 @@ def __init__( self.bucket_name = bucket_name self.object_name = object_name self.impersonation_chain = impersonation_chain - self.delegate_to = delegate_to def execute(self, context: Any) -> None: sheet_hook = GSheetsHook( gcp_conn_id=self.gcp_conn_id, impersonation_chain=self.impersonation_chain, - delegate_to=self.delegate_to, ) gcs_hook = GCSHook( gcp_conn_id=self.gcp_conn_id, diff --git a/providers/src/airflow/providers/google/suite/transfers/local_to_drive.py b/providers/src/airflow/providers/google/suite/transfers/local_to_drive.py index f2929bd5a1a0..70b07fec84d6 100644 --- a/providers/src/airflow/providers/google/suite/transfers/local_to_drive.py +++ b/providers/src/airflow/providers/google/suite/transfers/local_to_drive.py @@ -55,9 +55,6 @@ class LocalFilesystemToGoogleDriveOperator(BaseOperator): larger than 5MB, or to -1. :param resumable: True if this is a resumable upload. False means upload in a single request. - :param delegate_to: The account to impersonate using domain-wide delegation - of authority, if any. For this to work, the service account making the - request must have domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access token of the last account in the list, which will be impersonated @@ -87,7 +84,6 @@ def __init__( ignore_if_missing: bool = False, chunk_size: int = 100 * 1024 * 1024, resumable: bool = False, - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, folder_id: str = "root", show_full_target_path: bool = True, @@ -101,7 +97,6 @@ def __init__( self.ignore_if_missing = ignore_if_missing self.chunk_size = chunk_size self.resumable = resumable - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain self.folder_id = folder_id self.show_full_target_path = show_full_target_path @@ -109,7 +104,6 @@ def __init__( def execute(self, context: Context) -> list[str]: hook = GoogleDriveHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) diff --git a/providers/src/airflow/providers/google/suite/transfers/sql_to_sheets.py b/providers/src/airflow/providers/google/suite/transfers/sql_to_sheets.py index f38d9b230a1e..1e6bc3ca4b2e 100644 --- a/providers/src/airflow/providers/google/suite/transfers/sql_to_sheets.py +++ b/providers/src/airflow/providers/google/suite/transfers/sql_to_sheets.py @@ -37,9 +37,6 @@ class SQLToGoogleSheetsOperator(BaseSQLOperator): :param database: name of database which overwrite the defined one in connection :param spreadsheet_range: The A1 notation of the values to retrieve. :param gcp_conn_id: The connection ID to use when fetching connection info. - :param delegate_to: The account to impersonate using domain-wide delegation of authority, - if any. For this to work, the service account making the request must have - domain-wide delegation enabled. :param impersonation_chain: Optional service account to impersonate using short-term credentials, or chained list of accounts required to get the access_token of the last account in the list, which will be impersonated in the request. @@ -72,7 +69,6 @@ def __init__( database: str | None = None, spreadsheet_range: str = "Sheet1", gcp_conn_id: str = "google_cloud_default", - delegate_to: str | None = None, impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: @@ -85,7 +81,6 @@ def __init__( self.gcp_conn_id = gcp_conn_id self.spreadsheet_id = spreadsheet_id self.spreadsheet_range = spreadsheet_range - self.delegate_to = delegate_to self.impersonation_chain = impersonation_chain def _data_prep(self, data): @@ -117,7 +112,6 @@ def execute(self, context: Any) -> None: self.log.info("Connecting to Google") sheet_hook = GSheetsHook( gcp_conn_id=self.gcp_conn_id, - delegate_to=self.delegate_to, impersonation_chain=self.impersonation_chain, ) diff --git a/providers/tests/deprecations_ignore.yml b/providers/tests/deprecations_ignore.yml index b5aff3c84ea5..6e3adc6875fb 100644 --- a/providers/tests/deprecations_ignore.yml +++ b/providers/tests/deprecations_ignore.yml @@ -72,7 +72,6 @@ - providers/tests/google/cloud/hooks/test_life_sciences.py::TestLifeSciencesHookWithDefaultProjectIdFromConnection::test_life_science_client_creation - providers/tests/google/cloud/hooks/test_life_sciences.py::TestLifeSciencesHookWithDefaultProjectIdFromConnection::test_run_pipeline_immediately_complete - providers/tests/google/cloud/hooks/test_life_sciences.py::TestLifeSciencesHookWithDefaultProjectIdFromConnection::test_waiting_operation -- providers/tests/google/cloud/hooks/test_life_sciences.py::TestLifeSciencesHookWithPassedProjectId::test_delegate_to_runtime_error - providers/tests/google/cloud/hooks/test_life_sciences.py::TestLifeSciencesHookWithPassedProjectId::test_error_operation - providers/tests/google/cloud/hooks/test_life_sciences.py::TestLifeSciencesHookWithPassedProjectId::test_life_science_client_creation - providers/tests/google/cloud/hooks/test_life_sciences.py::TestLifeSciencesHookWithPassedProjectId::test_location_path @@ -90,16 +89,6 @@ - providers/tests/google/cloud/hooks/vertex_ai/test_custom_job.py::TestCustomJobWithoutDefaultProjectIdHook::test_delete_pipeline_job - providers/tests/google/cloud/hooks/vertex_ai/test_custom_job.py::TestCustomJobWithoutDefaultProjectIdHook::test_get_pipeline_job - providers/tests/google/cloud/hooks/vertex_ai/test_custom_job.py::TestCustomJobWithoutDefaultProjectIdHook::test_list_pipeline_jobs -- providers/tests/google/cloud/operators/test_dataproc.py::TestDataProcHadoopOperator::test_execute -- providers/tests/google/cloud/operators/test_dataproc.py::TestDataProcHiveOperator::test_builder -- providers/tests/google/cloud/operators/test_dataproc.py::TestDataProcHiveOperator::test_execute -- providers/tests/google/cloud/operators/test_dataproc.py::TestDataProcPigOperator::test_builder -- providers/tests/google/cloud/operators/test_dataproc.py::TestDataProcPigOperator::test_execute -- providers/tests/google/cloud/operators/test_dataproc.py::TestDataProcPySparkOperator::test_execute -- providers/tests/google/cloud/operators/test_dataproc.py::TestDataProcSparkOperator::test_execute -- providers/tests/google/cloud/operators/test_dataproc.py::TestDataProcSparkSqlOperator::test_builder -- providers/tests/google/cloud/operators/test_dataproc.py::TestDataProcSparkSqlOperator::test_execute -- providers/tests/google/cloud/operators/test_dataproc.py::TestDataProcSparkSqlOperator::test_execute_override_project_id - providers/tests/google/cloud/operators/test_dataproc.py::TestDataprocClusterScaleOperator::test_execute - providers/tests/google/cloud/operators/test_dataproc.py::test_create_cluster_operator_extra_links - providers/tests/google/cloud/operators/test_dataproc.py::test_scale_cluster_operator_extra_links diff --git a/providers/tests/google/cloud/hooks/test_automl.py b/providers/tests/google/cloud/hooks/test_automl.py index 26db131a9eef..c0b369283bd9 100644 --- a/providers/tests/google/cloud/hooks/test_automl.py +++ b/providers/tests/google/cloud/hooks/test_automl.py @@ -19,7 +19,6 @@ from unittest import mock -import pytest from google.api_core.gapic_v1.method import DEFAULT from google.cloud.automl_v1beta1 import AutoMlClient @@ -53,10 +52,6 @@ class TestAutoMLHook: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - CloudAutoMLHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") - def setup_method(self): with mock.patch( "airflow.providers.google.cloud.hooks.automl.GoogleBaseHook.__init__", diff --git a/providers/tests/google/cloud/hooks/test_bigquery.py b/providers/tests/google/cloud/hooks/test_bigquery.py index 02f442cfc7ca..b0e7f8efb209 100644 --- a/providers/tests/google/cloud/hooks/test_bigquery.py +++ b/providers/tests/google/cloud/hooks/test_bigquery.py @@ -69,18 +69,8 @@ def get_credentials_and_project_id(self): self.hook = MockedBigQueryHook() -def test_delegate_to_runtime_error(): - with pytest.raises(RuntimeError): - BigQueryHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") - - @pytest.mark.db_test class TestBigQueryHookMethods(_BigQueryBaseTestClass): - def test_credentials_path_derprecation(self): - with pytest.warns(AirflowProviderDeprecationWarning): - credentials_path = self.hook.credentials_path - assert credentials_path == "bigquery_hook_credentials.json" - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryConnection") @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook._authorize") @mock.patch("airflow.providers.google.cloud.hooks.bigquery.build") @@ -156,198 +146,6 @@ def test_get_pandas_df(self, mock_read_gbq): "select 1", credentials=CREDENTIALS, dialect="legacy", project_id=PROJECT_ID ) - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_service") - def test_invalid_schema_update_options(self, mock_get_service): - with pytest.raises( - ValueError, - match=( - r"\['THIS IS NOT VALID'\] contains invalid schema update options. " - r"Please only use one or more of the following options: " - r"\['ALLOW_FIELD_ADDITION', 'ALLOW_FIELD_RELAXATION'\]" - ), - ): - self.hook.run_load( - "test.test", - "test_schema.json", - ["test_data.json"], - schema_update_options=["THIS IS NOT VALID"], - ) - - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_service") - def test_invalid_schema_update_and_write_disposition(self, mock_get_service): - with pytest.raises( - ValueError, - match="schema_update_options is only allowed if" - " write_disposition is 'WRITE_APPEND' or 'WRITE_TRUNCATE'.", - ): - self.hook.run_load( - "test.test", - "test_schema.json", - ["test_data.json"], - schema_update_options=["ALLOW_FIELD_ADDITION"], - write_disposition="WRITE_EMPTY", - ) - - @mock.patch( - "airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.poll_job_complete", - side_effect=[False, True], - ) - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_client") - def test_cancel_queries(self, mock_client, mock_poll_job_complete): - running_job_id = 3 - - self.hook.running_job_id = running_job_id - self.hook.cancel_query() - - calls = [ - mock.call(job_id=running_job_id, project_id=PROJECT_ID, location=None), - mock.call(job_id=running_job_id, project_id=PROJECT_ID, location=None), - ] - mock_poll_job_complete.assert_has_calls(calls) - mock_client.assert_called_once_with(project_id=PROJECT_ID, location=None) - mock_client.return_value.cancel_job.assert_called_once_with(job_id=running_job_id) - - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_service") - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.insert_job") - def test_run_query_sql_dialect_default( - self, - mock_insert, - _, - ): - self.hook.run_query("query") - _, kwargs = mock_insert.call_args - assert kwargs["configuration"]["query"]["useLegacySql"] is True - - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_service") - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.insert_job") - def test_run_query_sql_dialect(self, mock_insert, _): - self.hook.run_query("query", use_legacy_sql=False) - _, kwargs = mock_insert.call_args - assert kwargs["configuration"]["query"]["useLegacySql"] is False - - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_service") - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.insert_job") - def test_run_query_sql_dialect_legacy_with_query_params(self, mock_insert, _): - params = [ - { - "name": "param_name", - "parameterType": {"type": "STRING"}, - "parameterValue": {"value": "param_value"}, - } - ] - self.hook.run_query("query", use_legacy_sql=False, query_params=params) - _, kwargs = mock_insert.call_args - assert kwargs["configuration"]["query"]["useLegacySql"] is False - - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_service") - def test_run_query_sql_dialect_legacy_with_query_params_fails(self, _): - params = [ - { - "name": "param_name", - "parameterType": {"type": "STRING"}, - "parameterValue": {"value": "param_value"}, - } - ] - with pytest.raises(ValueError, match="Query parameters are not allowed when using legacy SQL"): - self.hook.run_query("query", use_legacy_sql=True, query_params=params) - - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_service") - def test_run_query_without_sql_fails(self, _): - with pytest.raises( - TypeError, match=r"`BigQueryBaseCursor.run_query` missing 1 required positional argument: `sql`" - ): - self.hook.run_query(sql=None) - - @pytest.mark.parametrize( - "schema_update_options, write_disposition", - [ - (["ALLOW_FIELD_ADDITION"], "WRITE_APPEND"), - (["ALLOW_FIELD_RELAXATION"], "WRITE_APPEND"), - (["ALLOW_FIELD_ADDITION", "ALLOW_FIELD_RELAXATION"], "WRITE_APPEND"), - (["ALLOW_FIELD_ADDITION"], "WRITE_TRUNCATE"), - (["ALLOW_FIELD_RELAXATION"], "WRITE_TRUNCATE"), - (["ALLOW_FIELD_ADDITION", "ALLOW_FIELD_RELAXATION"], "WRITE_TRUNCATE"), - ], - ) - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_service") - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.insert_job") - def test_run_query_schema_update_options( - self, - mock_insert, - _, - schema_update_options, - write_disposition, - ): - self.hook.run_query( - sql="query", - destination_dataset_table="my_dataset.my_table", - schema_update_options=schema_update_options, - write_disposition=write_disposition, - ) - _, kwargs = mock_insert.call_args - assert kwargs["configuration"]["query"]["schemaUpdateOptions"] == schema_update_options - assert kwargs["configuration"]["query"]["writeDisposition"] == write_disposition - - @pytest.mark.parametrize( - "schema_update_options, write_disposition, expected_regex", - [ - ( - ["INCORRECT_OPTION"], - None, - r"\['INCORRECT_OPTION'\] contains invalid schema update options\. " - r"Please only use one or more of the following options: " - r"\['ALLOW_FIELD_ADDITION', 'ALLOW_FIELD_RELAXATION'\]", - ), - ( - ["ALLOW_FIELD_ADDITION", "ALLOW_FIELD_RELAXATION", "INCORRECT_OPTION"], - None, - r"\['ALLOW_FIELD_ADDITION', 'ALLOW_FIELD_RELAXATION', 'INCORRECT_OPTION'\] contains invalid " - r"schema update options\. Please only use one or more of the following options: " - r"\['ALLOW_FIELD_ADDITION', 'ALLOW_FIELD_RELAXATION'\]", - ), - ( - ["ALLOW_FIELD_ADDITION"], - None, - r"schema_update_options is only allowed if write_disposition is " - r"'WRITE_APPEND' or 'WRITE_TRUNCATE'", - ), - ], - ) - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_service") - def test_run_query_schema_update_options_incorrect( - self, _, schema_update_options, write_disposition, expected_regex - ): - with pytest.raises(ValueError, match=expected_regex): - self.hook.run_query( - sql="query", - destination_dataset_table="my_dataset.my_table", - schema_update_options=schema_update_options, - write_disposition=write_disposition, - ) - - @pytest.mark.parametrize("bool_val", [True, False]) - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_service") - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.insert_job") - def test_api_resource_configs(self, mock_insert, _, bool_val): - self.hook.run_query("query", api_resource_configs={"query": {"useQueryCache": bool_val}}) - _, kwargs = mock_insert.call_args - assert kwargs["configuration"]["query"]["useQueryCache"] is bool_val - assert kwargs["configuration"]["query"]["useLegacySql"] is True - - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_service") - def test_api_resource_configs_duplication_warning(self, mock_get_service): - with pytest.raises( - ValueError, - match=( - r"Values of useLegacySql param are duplicated\. api_resource_configs " - r"contained useLegacySql param in `query` config and useLegacySql was " - r"also provided with arg to run_query\(\) method\. Please remove duplicates\." - ), - ): - self.hook.run_query( - "query", use_legacy_sql=True, api_resource_configs={"query": {"useLegacySql": False}} - ) - def test_validate_value(self): with pytest.raises( TypeError, match="case_1 argument must have a type not " @@ -392,44 +190,6 @@ def test_validate_src_fmt_configs(self): "compatibility_val" in src_fmt_configs ), "_validate_src_fmt_configs should add backward_compatibility config" - @pytest.mark.parametrize("fmt", ["AVRO", "PARQUET", "NEWLINE_DELIMITED_JSON", "DATASTORE_BACKUP"]) - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.insert_job") - def test_run_load_with_non_csv_as_src_fmt(self, _, fmt): - try: - self.hook.run_load( - destination_project_dataset_table="my_dataset.my_table", - source_uris=[], - source_format=fmt, - autodetect=True, - ) - except ValueError as ex: - pytest.fail("run_load() raised ValueError unexpectedly!", ex) - - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.insert_job") - def test_run_extract(self, mock_insert): - source_project_dataset_table = f"{PROJECT_ID}.{DATASET_ID}.{TABLE_ID}" - destination_cloud_storage_uris = ["gs://bucket/file.csv"] - expected_configuration = { - "extract": { - "sourceTable": { - "projectId": PROJECT_ID, - "datasetId": DATASET_ID, - "tableId": TABLE_ID, - }, - "compression": "NONE", - "destinationUris": destination_cloud_storage_uris, - "destinationFormat": "CSV", - "fieldDelimiter": ",", - "printHeader": True, - } - } - - self.hook.run_extract( - source_project_dataset_table=source_project_dataset_table, - destination_cloud_storage_uris=destination_cloud_storage_uris, - ) - mock_insert.assert_called_once_with(configuration=expected_configuration, project_id=PROJECT_ID) - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.Table") @mock.patch("airflow.providers.google.cloud.hooks.bigquery.SchemaField") @mock.patch("airflow.providers.google.cloud.hooks.bigquery.Client") @@ -492,14 +252,6 @@ def test_list_rows_with_empty_selected_fields(self, mock_client, mock_table): mock_table.from_api_repr.reset_mock() mock_client.return_value.list_rows.reset_mock() - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.Client") - def test_run_table_delete(self, mock_client): - source_dataset_table = f"{DATASET_ID}.{TABLE_ID}" - self.hook.run_table_delete(source_dataset_table, ignore_if_missing=False) - mock_client.return_value.delete_table.assert_called_once_with( - table=source_dataset_table, not_found_ok=False - ) - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.create_empty_table") @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_dataset_tables") def test_table_upsert_create_new_table(self, mock_get, mock_create): @@ -571,25 +323,6 @@ def test_run_grant_dataset_view_access_already_granted(self, mock_update, mock_g mock_get.assert_called_once_with(project_id=PROJECT_ID, dataset_id=DATASET_ID) assert len(mock_update.calls) == 0 - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.Client") - def test_get_dataset_tables_list(self, mock_client): - table_list = [ - {"projectId": PROJECT_ID, "datasetId": DATASET_ID, "tableId": "a-1"}, - {"projectId": PROJECT_ID, "datasetId": DATASET_ID, "tableId": "b-1"}, - {"projectId": PROJECT_ID, "datasetId": DATASET_ID, "tableId": "a-2"}, - {"projectId": PROJECT_ID, "datasetId": DATASET_ID, "tableId": "b-2"}, - ] - table_list_response = [Table.from_api_repr({"tableReference": t}) for t in table_list] - mock_client.return_value.list_tables.return_value = table_list_response - - dataset_reference = DatasetReference(PROJECT_ID, DATASET_ID) - result = self.hook.get_dataset_tables_list(dataset_id=DATASET_ID, project_id=PROJECT_ID) - - mock_client.return_value.list_tables.assert_called_once_with( - dataset=dataset_reference, max_results=None - ) - assert table_list == result - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_client") def test_poll_job_complete(self, mock_client): self.hook.poll_job_complete(job_id=JOB_ID, location=LOCATION, project_id=PROJECT_ID) @@ -628,65 +361,6 @@ def test_get_job_credentials_error(self, mock_client, error): with pytest.raises(type(error)): self.hook.get_job(job_id=JOB_ID, location=LOCATION, project_id=PROJECT_ID) - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.poll_job_complete") - @mock.patch("logging.Logger.info") - def test_cancel_query_jobs_to_cancel( - self, - mock_logger_info, - poll_job_complete, - ): - poll_job_complete.return_value = True - - self.hook.running_job_id = JOB_ID - self.hook.cancel_query() - poll_job_complete.assert_called_once_with(job_id=JOB_ID, project_id=PROJECT_ID, location=None) - mock_logger_info.has_call(mock.call("No running BigQuery jobs to cancel.")) - - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_client") - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.poll_job_complete") - @mock.patch("time.sleep") - @mock.patch("logging.Logger.info") - def test_cancel_query_cancel_timeout( - self, - mock_logger_info, - mock_sleep, - poll_job_complete, - mock_client, - ): - poll_job_complete.side_effect = [False] * 13 - - self.hook.running_job_id = JOB_ID - self.hook.cancel_query() - mock_client.return_value.cancel_job.assert_called_once_with(job_id=JOB_ID) - assert poll_job_complete.call_count == 13 - assert mock_sleep.call_count == 11 - mock_logger_info.has_call( - mock.call( - f"Stopping polling due to timeout. Job with id {JOB_ID} " - "has not completed cancel and may or may not finish." - ) - ) - - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_client") - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.poll_job_complete") - @mock.patch("time.sleep") - @mock.patch("logging.Logger.info") - def test_cancel_query_cancel_completed( - self, - mock_logger_info, - mock_sleep, - poll_job_complete, - mock_client, - ): - poll_job_complete.side_effect = [False] * 12 + [True] - - self.hook.running_job_id = JOB_ID - self.hook.cancel_query() - mock_client.return_value.cancel_job.assert_called_once_with(job_id=JOB_ID) - assert poll_job_complete.call_count == 13 - assert mock_sleep.call_count == 11 - mock_logger_info.has_call(mock.call(f"Job successfully canceled: {PROJECT_ID}, {PROJECT_ID}")) - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.Client") def test_get_schema(self, mock_client): table = { @@ -878,15 +552,6 @@ def test_update_table_schema_without_policy_tags(self, mock_update, mock_get_sch fields=["schema"], ) - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_service") - def test_invalid_source_format(self, mock_get_service): - with pytest.raises( - ValueError, - match=r"JSON is not a valid source format. Please use one of the following types: \['CSV', " - r"'NEWLINE_DELIMITED_JSON', 'AVRO', 'GOOGLE_SHEETS', 'DATASTORE_BACKUP', 'PARQUET'\]", - ): - self.hook.run_load("test.test", "test_schema.json", ["test_data.json"], source_format="json") - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.Client") def test_insert_all_succeed(self, mock_client): rows = [{"json": {"a_key": "a_value_0"}}] @@ -917,17 +582,6 @@ def test_insert_all_fail(self, mock_client): project_id=PROJECT_ID, dataset_id=DATASET_ID, table_id=TABLE_ID, rows=rows, fail_on_error=True ) - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.insert_job") - def test_run_query_with_arg(self, mock_insert): - self.hook.run_query( - sql="select 1", - destination_dataset_table="my_dataset.my_table", - labels={"label1": "test1", "label2": "test2"}, - ) - - _, kwargs = mock_insert.call_args - assert kwargs["configuration"]["labels"] == {"label1": "test1", "label2": "test2"} - @pytest.mark.parametrize("nowait", [True, False]) @mock.patch("airflow.providers.google.cloud.hooks.bigquery.QueryJob") @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_client") @@ -1161,58 +815,6 @@ def test_create_view(self, mock_bq_client, mock_table): retry=DEFAULT_RETRY, ) - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.Table") - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.Client") - def test_patch_table(self, mock_client, mock_table): - description_patched = "Test description." - expiration_time_patched = 2524608000000 - friendly_name_patched = "Test friendly name." - labels_patched = {"label1": "test1", "label2": "test2"} - schema_patched = [ - {"name": "id", "type": "STRING", "mode": "REQUIRED"}, - {"name": "name", "type": "STRING", "mode": "NULLABLE"}, - {"name": "balance", "type": "FLOAT", "mode": "NULLABLE"}, - {"name": "new_field", "type": "STRING", "mode": "NULLABLE"}, - ] - time_partitioning_patched = {"expirationMs": 10000000} - require_partition_filter_patched = True - view_patched = { - "query": "SELECT * FROM `test-project-id.test_dataset_id.test_table_prefix*` LIMIT 500", - "useLegacySql": False, - } - - self.hook.patch_table( - dataset_id=DATASET_ID, - table_id=TABLE_ID, - project_id=PROJECT_ID, - description=description_patched, - expiration_time=expiration_time_patched, - friendly_name=friendly_name_patched, - labels=labels_patched, - schema=schema_patched, - time_partitioning=time_partitioning_patched, - require_partition_filter=require_partition_filter_patched, - view=view_patched, - ) - - body = { - "description": description_patched, - "expirationTime": expiration_time_patched, - "friendlyName": friendly_name_patched, - "labels": labels_patched, - "schema": {"fields": schema_patched}, - "timePartitioning": time_partitioning_patched, - "view": view_patched, - "requirePartitionFilter": require_partition_filter_patched, - } - fields = list(body.keys()) - body["tableReference"] = TABLE_REFERENCE_REPR - - mock_table.from_api_repr.assert_called_once_with(body) - mock_client.return_value.update_table.assert_called_once_with( - table=mock_table.from_api_repr.return_value, fields=fields - ) - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.Table") @mock.patch("airflow.providers.google.cloud.hooks.bigquery.Client") def test_create_empty_table_succeed(self, mock_bq_client, mock_table): @@ -1432,38 +1034,38 @@ def test_description_no_schema(self, mock_insert, mock_build): bq_cursor.execute("UPDATE airflow.test_table SET foo = 'bar'") assert bq_cursor.description == [] - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_service") - def test_close(self, mock_get_service): + @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_client") + def test_close(self, mock_get_client): bq_cursor = self.hook.get_cursor() result = bq_cursor.close() assert result is None - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_service") - def test_rowcount(self, mock_get_service): + @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_client") + def test_rowcount(self, mock_get_client): bq_cursor = self.hook.get_cursor() result = bq_cursor.rowcount assert -1 == result - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_service") + @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_client") @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryCursor.next") - def test_fetchone(self, mock_next, mock_get_service): + def test_fetchone(self, mock_next, mock_get_client): bq_cursor = self.hook.get_cursor() result = bq_cursor.fetchone() mock_next.call_count == 1 assert mock_next.return_value == result - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_service") + @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_client") @mock.patch( "airflow.providers.google.cloud.hooks.bigquery.BigQueryCursor.fetchone", side_effect=[1, 2, 3, None] ) - def test_fetchall(self, mock_fetchone, mock_get_service): + def test_fetchall(self, mock_fetchone, mock_get_client): bq_cursor = self.hook.get_cursor() result = bq_cursor.fetchall() assert [1, 2, 3] == result - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_service") + @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_client") @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryCursor.fetchone") - def test_fetchmany(self, mock_fetchone, mock_get_service): + def test_fetchmany(self, mock_fetchone, mock_get_client): side_effect_values = [1, 2, 3, None] bq_cursor = self.hook.get_cursor() mock_fetchone.side_effect = side_effect_values @@ -1478,15 +1080,15 @@ def test_fetchmany(self, mock_fetchone, mock_get_service): result = bq_cursor.fetchmany(5) assert [1, 2, 3] == result - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_service") - def test_next_no_jobid(self, mock_get_service): + @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_client") + def test_next_no_jobid(self, mock_get_client): bq_cursor = self.hook.get_cursor() bq_cursor.job_id = None result = bq_cursor.next() assert result is None - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_service") - def test_next_buffer(self, mock_get_service): + @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_client") + def test_next_buffer(self, mock_get_client): bq_cursor = self.hook.get_cursor() bq_cursor.job_id = JOB_ID bq_cursor.buffer = [1, 2] @@ -1558,8 +1160,8 @@ def test_flush_cursor_in_execute(self, _, mock_insert, mock_build): bq_cursor.execute("SELECT %(foo)s", {"foo": "bar"}) assert mock_insert.call_count == 1 - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_service") - def test_flush_cursor(self, mock_get_service): + @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_client") + def test_flush_cursor(self, mock_get_client): bq_cursor = self.hook.get_cursor() bq_cursor.page_token = "456dcea9-fcbf-4f02-b570-83f5297c685e" bq_cursor.job_id = "c0a79ae4-0e72-4593-a0d0-7dbbf726f193" @@ -1571,8 +1173,8 @@ def test_flush_cursor(self, mock_get_service): assert not bq_cursor.all_pages_loaded assert bq_cursor.buffer == [] - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_service") - def test_arraysize(self, mock_get_service): + @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_client") + def test_arraysize(self, mock_get_client): bq_cursor = self.hook.get_cursor() assert bq_cursor.buffersize is None assert bq_cursor.arraysize == 1 @@ -1736,17 +1338,6 @@ def test_delete_dataset(self, mock_client): not_found_ok=True, ) - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_service") - def test_patch_dataset(self, mock_get_service): - dataset_resource = {"access": [{"role": "WRITER", "groupByEmail": "cloud-logs@google.com"}]} - - method = mock_get_service.return_value.datasets.return_value.patch - self.hook.patch_dataset( - dataset_id=DATASET_ID, project_id=PROJECT_ID, dataset_resource=dataset_resource - ) - - method.assert_called_once_with(projectId=PROJECT_ID, datasetId=DATASET_ID, body=dataset_resource) - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.Dataset") @mock.patch("airflow.providers.google.cloud.hooks.bigquery.Client") def test_update_dataset(self, mock_client, mock_dataset): @@ -1780,77 +1371,6 @@ def test_update_dataset(self, mock_client, mock_dataset): @pytest.mark.db_test class TestTimePartitioningInRunJob(_BigQueryBaseTestClass): - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.insert_job") - def test_run_load_default(self, mock_insert): - self.hook.run_load( - destination_project_dataset_table="my_dataset.my_table", - schema_fields=[], - source_uris=[], - ) - - _, kwargs = mock_insert.call_args - assert kwargs["configuration"]["load"].get("timePartitioning") is None - - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.insert_job") - def test_run_with_auto_detect(self, mock_insert): - destination_project_dataset_table = "autodetect.table" - self.hook.run_load(destination_project_dataset_table, [], [], autodetect=True) - _, kwargs = mock_insert.call_args - assert kwargs["configuration"]["load"]["autodetect"] is True - - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.insert_job") - def test_run_load_with_arg(self, mock_insert): - self.hook.run_load( - destination_project_dataset_table=f"{DATASET_ID}.{TABLE_ID}", - schema_fields=[], - source_uris=[], - time_partitioning={"type": "DAY", "field": "test_field", "expirationMs": 1000}, - ) - configuration = { - "load": { - "autodetect": False, - "createDisposition": "CREATE_IF_NEEDED", - "destinationTable": {"projectId": PROJECT_ID, "datasetId": DATASET_ID, "tableId": TABLE_ID}, - "sourceFormat": "CSV", - "sourceUris": [], - "writeDisposition": "WRITE_EMPTY", - "ignoreUnknownValues": False, - "timePartitioning": {"type": "DAY", "field": "test_field", "expirationMs": 1000}, - "skipLeadingRows": 0, - "fieldDelimiter": ",", - "quote": None, - "allowQuotedNewlines": False, - "encoding": "UTF-8", - } - } - mock_insert.assert_called_once_with(configuration=configuration, project_id=PROJECT_ID) - - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.insert_job") - def test_run_query_with_arg(self, mock_insert): - self.hook.run_query( - sql="select 1", - destination_dataset_table=f"{DATASET_ID}.{TABLE_ID}", - priority="BATCH", - time_partitioning={"type": "DAY", "field": "test_field", "expirationMs": 1000}, - ) - - configuration = { - "query": { - "query": "select 1", - "priority": "BATCH", - "useLegacySql": True, - "timePartitioning": {"type": "DAY", "field": "test_field", "expirationMs": 1000}, - "schemaUpdateOptions": [], - "destinationTable": {"projectId": PROJECT_ID, "datasetId": DATASET_ID, "tableId": TABLE_ID}, - "allowLargeResults": False, - "flattenResults": None, - "writeDisposition": "WRITE_EMPTY", - "createDisposition": "CREATE_IF_NEEDED", - } - } - - mock_insert.assert_called_once_with(configuration=configuration, project_id=PROJECT_ID, location=None) - def test_dollar_makes_partition(self): tp_out = _cleanse_time_partitioning("test.teast$20170101", {}) expect = {"type": "DAY"} @@ -1865,52 +1385,6 @@ def test_extra_time_partitioning_options(self): assert tp_out == expect -@pytest.mark.db_test -class TestClusteringInRunJob(_BigQueryBaseTestClass): - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.insert_job") - def test_run_load_default(self, mock_insert): - self.hook.run_load( - destination_project_dataset_table="my_dataset.my_table", - schema_fields=[], - source_uris=[], - ) - - _, kwargs = mock_insert.call_args - assert kwargs["configuration"]["load"].get("clustering") is None - - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.insert_job") - def test_run_load_with_arg(self, mock_insert): - self.hook.run_load( - destination_project_dataset_table="my_dataset.my_table", - schema_fields=[], - source_uris=[], - cluster_fields=["field1", "field2"], - time_partitioning={"type": "DAY"}, - ) - - _, kwargs = mock_insert.call_args - assert kwargs["configuration"]["load"]["clustering"] == {"fields": ["field1", "field2"]} - - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.insert_job") - def test_run_query_default(self, mock_insert): - self.hook.run_query(sql="select 1") - - _, kwargs = mock_insert.call_args - assert kwargs["configuration"]["query"].get("clustering") is None - - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.insert_job") - def test_run_query_with_arg(self, mock_insert): - self.hook.run_query( - sql="select 1", - destination_dataset_table="my_dataset.my_table", - cluster_fields=["field1", "field2"], - time_partitioning={"type": "DAY"}, - ) - - _, kwargs = mock_insert.call_args - assert kwargs["configuration"]["query"]["clustering"] == {"fields": ["field1", "field2"]} - - @pytest.mark.db_test class TestBigQueryHookLegacySql(_BigQueryBaseTestClass): """Ensure `use_legacy_sql` param in `BigQueryHook` propagates properly.""" @@ -1941,29 +1415,6 @@ def test_legacy_sql_override_propagates_properly( assert kwargs["configuration"]["query"]["useLegacySql"] is False -@pytest.mark.db_test -class TestBigQueryHookRunWithConfiguration(_BigQueryBaseTestClass): - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.LoadJob") - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.get_client") - def test_run_with_configuration_location(self, mock_client, mock_job): - running_job_id = "job_vjdi28vskdui2onru23" - location = "asia-east1" - mock_job._JOB_TYPE = "load" - - conf = {"load": {}} - self.hook.running_job_id = running_job_id - self.hook.location = location - self.hook.run_with_configuration(conf) - mock_client.assert_called_once_with(project_id=PROJECT_ID, location=location) - mock_job.from_api_repr.assert_called_once_with( - { - "configuration": conf, - "jobReference": {"jobId": mock.ANY, "projectId": PROJECT_ID, "location": location}, - }, - mock_client.return_value, - ) - - @pytest.mark.db_test class TestBigQueryWithKMS(_BigQueryBaseTestClass): @mock.patch("airflow.providers.google.cloud.hooks.bigquery.Table") @@ -1992,85 +1443,6 @@ def test_create_empty_table_with_kms(self, mock_bq_client, mock_table): retry=DEFAULT_RETRY, ) - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.create_empty_table") - def test_create_external_table_with_kms(self, mock_create): - external_project_dataset_table = f"{PROJECT_ID}.{DATASET_ID}.{TABLE_ID}" - source_uris = ["test_data.csv"] - source_format = "CSV" - autodetect = False - compression = "NONE" - ignore_unknown_values = False - max_bad_records = 10 - skip_leading_rows = 1 - field_delimiter = "," - quote_character = None - allow_quoted_newlines = False - allow_jagged_rows = False - encoding = "UTF-8" - labels = {"label1": "test1", "label2": "test2"} - schema_fields = [ - { - "mode": "REQUIRED", - "name": "id", - "type": "STRING", - "description": None, - "policyTags": {"names": []}, - } - ] - encryption_configuration = {"kms_key_name": "projects/p/locations/l/keyRings/k/cryptoKeys/c"} - - self.hook.create_external_table( - external_project_dataset_table=external_project_dataset_table, - source_uris=source_uris, - source_format=source_format, - autodetect=autodetect, - compression=compression, - ignore_unknown_values=ignore_unknown_values, - max_bad_records=max_bad_records, - skip_leading_rows=skip_leading_rows, - field_delimiter=field_delimiter, - quote_character=quote_character, - allow_jagged_rows=allow_jagged_rows, - encoding=encoding, - allow_quoted_newlines=allow_quoted_newlines, - labels=labels, - schema_fields=schema_fields, - encryption_configuration=encryption_configuration, - ) - - body = { - "externalDataConfiguration": { - "autodetect": autodetect, - "sourceFormat": source_format, - "sourceUris": source_uris, - "compression": compression, - "ignoreUnknownValues": ignore_unknown_values, - "schema": {"fields": schema_fields}, - "maxBadRecords": max_bad_records, - "csvOptions": { - "skipLeadingRows": skip_leading_rows, - "fieldDelimiter": field_delimiter, - "quote": quote_character, - "allowQuotedNewlines": allow_quoted_newlines, - "allowJaggedRows": allow_jagged_rows, - "encoding": encoding, - }, - }, - "tableReference": { - "projectId": PROJECT_ID, - "datasetId": DATASET_ID, - "tableId": TABLE_ID, - }, - "labels": labels, - "encryptionConfiguration": encryption_configuration, - } - mock_create.assert_called_once_with( - table_resource=body, - project_id=PROJECT_ID, - location=None, - exists_ok=True, - ) - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.Table") @mock.patch("airflow.providers.google.cloud.hooks.bigquery.Client") def test_update_table(self, mock_client, mock_table): @@ -2123,99 +1495,8 @@ def test_update_table(self, mock_client, mock_table): table=mock_table.from_api_repr.return_value, fields=fields ) - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.insert_job") - def test_run_query_with_kms(self, mock_insert): - encryption_configuration = {"kms_key_name": "projects/p/locations/l/keyRings/k/cryptoKeys/c"} - self.hook.run_query(sql="query", encryption_configuration=encryption_configuration) - _, kwargs = mock_insert.call_args - assert ( - kwargs["configuration"]["query"]["destinationEncryptionConfiguration"] is encryption_configuration - ) - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.insert_job") - def test_run_copy_with_kms(self, mock_insert): - encryption_configuration = {"kms_key_name": "projects/p/locations/l/keyRings/k/cryptoKeys/c"} - self.hook.run_copy( - source_project_dataset_tables="p.d.st", - destination_project_dataset_table="p.d.dt", - encryption_configuration=encryption_configuration, - ) - _, kwargs = mock_insert.call_args - assert ( - kwargs["configuration"]["copy"]["destinationEncryptionConfiguration"] is encryption_configuration - ) - - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.insert_job") - def test_run_load_with_kms(self, mock_insert): - encryption_configuration = {"kms_key_name": "projects/p/locations/l/keyRings/k/cryptoKeys/c"} - self.hook.run_load( - destination_project_dataset_table="p.d.dt", - source_uris=["abc.csv"], - autodetect=True, - encryption_configuration=encryption_configuration, - ) - _, kwargs = mock_insert.call_args - assert ( - kwargs["configuration"]["load"]["destinationEncryptionConfiguration"] is encryption_configuration - ) - - -@pytest.mark.db_test -class TestBigQueryWithLabelsAndDescription(_BigQueryBaseTestClass): - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.insert_job") - def test_run_load_labels(self, mock_insert): - labels = {"label1": "test1", "label2": "test2"} - self.hook.run_load( - destination_project_dataset_table="my_dataset.my_table", - schema_fields=[], - source_uris=[], - labels=labels, - ) - - _, kwargs = mock_insert.call_args - assert kwargs["configuration"]["load"]["destinationTableProperties"]["labels"] is labels - - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.insert_job") - def test_run_load_description(self, mock_insert): - description = "Test Description" - self.hook.run_load( - destination_project_dataset_table="my_dataset.my_table", - schema_fields=[], - source_uris=[], - description=description, - ) - - _, kwargs = mock_insert.call_args - assert kwargs["configuration"]["load"]["destinationTableProperties"]["description"] is description - - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.create_empty_table") - def test_create_external_table_labels(self, mock_create): - labels = {"label1": "test1", "label2": "test2"} - self.hook.create_external_table( - external_project_dataset_table="my_dataset.my_table", - schema_fields=[], - source_uris=[], - labels=labels, - ) - - _, kwargs = mock_create.call_args - assert kwargs["table_resource"]["labels"] == labels - - @mock.patch("airflow.providers.google.cloud.hooks.bigquery.BigQueryHook.create_empty_table") - def test_create_external_table_description(self, mock_create): - description = "Test Description" - self.hook.create_external_table( - external_project_dataset_table="my_dataset.my_table", - schema_fields=[], - source_uris=[], - description=description, - ) - - _, kwargs = mock_create.call_args - assert kwargs["table_resource"]["description"] is description - - -class _BigQueryBaseAsyncTestClass: +class TestBigQueryAsyncHookMethods: def setup_method(self) -> None: class MockedBigQueryAsyncHook(BigQueryAsyncHook): def get_credentials_and_project_id(self): @@ -2223,8 +1504,6 @@ def get_credentials_and_project_id(self): self.hook = MockedBigQueryAsyncHook() - -class TestBigQueryAsyncHookMethods(_BigQueryBaseAsyncTestClass): @pytest.mark.db_test @pytest.mark.asyncio @mock.patch("google.auth.default") diff --git a/providers/tests/google/cloud/hooks/test_bigquery_dts.py b/providers/tests/google/cloud/hooks/test_bigquery_dts.py index 163ca25273b1..654c6fa65a17 100644 --- a/providers/tests/google/cloud/hooks/test_bigquery_dts.py +++ b/providers/tests/google/cloud/hooks/test_bigquery_dts.py @@ -57,10 +57,6 @@ class TestBigQueryDataTransferHook: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - BiqQueryDataTransferServiceHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") - def setup_method(self): with mock.patch( "airflow.providers.google.cloud.hooks.bigquery_dts.GoogleBaseHook.__init__", @@ -134,10 +130,6 @@ def test_get_transfer_run(self, service_mock): class TestAsyncBiqQueryDataTransferServiceHook: HOOK_MODULE_PATH = "airflow.providers.google.cloud.hooks.bigquery_dts" - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - AsyncBiqQueryDataTransferServiceHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") - @pytest.fixture def mock_client(self): with mock.patch( diff --git a/providers/tests/google/cloud/hooks/test_bigtable.py b/providers/tests/google/cloud/hooks/test_bigtable.py index f1929ba00d58..738a7941a9e9 100644 --- a/providers/tests/google/cloud/hooks/test_bigtable.py +++ b/providers/tests/google/cloud/hooks/test_bigtable.py @@ -21,7 +21,6 @@ from unittest.mock import PropertyMock import google -import pytest from google.cloud.bigtable import Client, enums from google.cloud.bigtable.instance import Instance @@ -51,10 +50,6 @@ class TestBigtableHookNoDefaultProjectId: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - BigtableHook(gcp_conn_id="test", delegate_to="delegate_to") - def setup_method(self): with mock.patch( "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.__init__", diff --git a/providers/tests/google/cloud/hooks/test_cloud_build.py b/providers/tests/google/cloud/hooks/test_cloud_build.py index e65a5cdb8ba3..a4762b440b2d 100644 --- a/providers/tests/google/cloud/hooks/test_cloud_build.py +++ b/providers/tests/google/cloud/hooks/test_cloud_build.py @@ -61,10 +61,6 @@ class TestCloudBuildHook: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - CloudBuildHook(gcp_conn_id="test", delegate_to="delegate_to") - def setup_method(self): with mock.patch( "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.__init__", @@ -353,10 +349,6 @@ def test_update_build_trigger(self, get_conn): @pytest.mark.db_test class TestAsyncHook: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - CloudBuildAsyncHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") - @pytest.fixture def hook(self): return CloudBuildAsyncHook( diff --git a/providers/tests/google/cloud/hooks/test_cloud_composer.py b/providers/tests/google/cloud/hooks/test_cloud_composer.py index 945a63760552..a7ef0e347fe4 100644 --- a/providers/tests/google/cloud/hooks/test_cloud_composer.py +++ b/providers/tests/google/cloud/hooks/test_cloud_composer.py @@ -65,10 +65,6 @@ def mock_init(*args, **kwargs): class TestCloudComposerHook: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - CloudComposerHook(gcp_conn_id="test", delegate_to="delegate_to") - def setup_method(self): with mock.patch(BASE_STRING.format("GoogleBaseHook.__init__"), new=mock_init): self.hook = CloudComposerHook(gcp_conn_id="test") @@ -263,10 +259,6 @@ def test_poll_airflow_command(self, mock_client) -> None: class TestCloudComposerAsyncHook: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - CloudComposerAsyncHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") - def setup_method(self, method): with mock.patch(BASE_STRING.format("GoogleBaseHook.__init__"), new=mock_init): self.hook = CloudComposerAsyncHook(gcp_conn_id="test") diff --git a/providers/tests/google/cloud/hooks/test_cloud_memorystore.py b/providers/tests/google/cloud/hooks/test_cloud_memorystore.py index d96ebc5b4322..78b274932e0f 100644 --- a/providers/tests/google/cloud/hooks/test_cloud_memorystore.py +++ b/providers/tests/google/cloud/hooks/test_cloud_memorystore.py @@ -58,10 +58,6 @@ class TestCloudMemorystoreWithDefaultProjectIdHook: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - CloudMemorystoreHook(gcp_conn_id="test", delegate_to="delegate_to") - def setup_method(self): with mock.patch( "airflow.providers.google.cloud.hooks.cloud_memorystore.CloudMemorystoreHook.__init__", diff --git a/providers/tests/google/cloud/hooks/test_cloud_sql.py b/providers/tests/google/cloud/hooks/test_cloud_sql.py index 3365dc76aec3..13e9d3f57469 100644 --- a/providers/tests/google/cloud/hooks/test_cloud_sql.py +++ b/providers/tests/google/cloud/hooks/test_cloud_sql.py @@ -74,10 +74,6 @@ def session(): class TestGcpSqlHookDefaultProjectId: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - CloudSQLHook(api_version="v1", gcp_conn_id="test", delegate_to="delegate_to") - def setup_method(self): with mock.patch( "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.__init__", diff --git a/providers/tests/google/cloud/hooks/test_cloud_storage_transfer_service.py b/providers/tests/google/cloud/hooks/test_cloud_storage_transfer_service.py index a68bcd3afef2..1a490dc9ca99 100644 --- a/providers/tests/google/cloud/hooks/test_cloud_storage_transfer_service.py +++ b/providers/tests/google/cloud/hooks/test_cloud_storage_transfer_service.py @@ -104,10 +104,6 @@ def _with_name(body, job_name): class TestGCPTransferServiceHookWithPassedName: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - CloudDataTransferServiceHook(gcp_conn_id="test", delegate_to="delegate_to") - def setup_method(self): with mock.patch( "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.__init__", diff --git a/providers/tests/google/cloud/hooks/test_compute.py b/providers/tests/google/cloud/hooks/test_compute.py index 24a43ac3d87b..ac0b288ea624 100644 --- a/providers/tests/google/cloud/hooks/test_compute.py +++ b/providers/tests/google/cloud/hooks/test_compute.py @@ -56,10 +56,6 @@ class TestGcpComputeHookApiCall: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - ComputeEngineHook(api_version=API_VERSION, gcp_conn_id=GCP_CONN_ID, delegate_to="delegate_to") - def setup_method(self): with mock.patch( BASE_STRING.format("GoogleBaseHook.__init__"), diff --git a/providers/tests/google/cloud/hooks/test_compute_ssh.py b/providers/tests/google/cloud/hooks/test_compute_ssh.py index dfcd0d719c0a..6065ba0eacce 100644 --- a/providers/tests/google/cloud/hooks/test_compute_ssh.py +++ b/providers/tests/google/cloud/hooks/test_compute_ssh.py @@ -45,10 +45,6 @@ class TestComputeEngineHookWithPassedProjectId: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - ComputeEngineSSHHook(gcp_conn_id="gcpssh", delegate_to="delegate_to") - def test_os_login_hook(self, mocker): mock_os_login_hook = mocker.patch.object(OSLoginHook, "__init__", return_value=None, spec=OSLoginHook) diff --git a/providers/tests/google/cloud/hooks/test_datacatalog.py b/providers/tests/google/cloud/hooks/test_datacatalog.py index 29ce515e8886..f535e0b36bc8 100644 --- a/providers/tests/google/cloud/hooks/test_datacatalog.py +++ b/providers/tests/google/cloud/hooks/test_datacatalog.py @@ -85,10 +85,6 @@ class TestCloudDataCatalog: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - CloudDataCatalogHook(gcp_conn_id="test", delegate_to="delegate_to") - def setup_method(self): with mock.patch( "airflow.providers.google.cloud.hooks.datacatalog.CloudDataCatalogHook.__init__", diff --git a/providers/tests/google/cloud/hooks/test_dataflow.py b/providers/tests/google/cloud/hooks/test_dataflow.py index 4e1aa6028779..eb69952e47cb 100644 --- a/providers/tests/google/cloud/hooks/test_dataflow.py +++ b/providers/tests/google/cloud/hooks/test_dataflow.py @@ -210,10 +210,6 @@ def test_fn(self, *args, **kwargs): @pytest.mark.db_test class TestDataflowHook: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - DataflowHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") - def setup_method(self): self.dataflow_hook = DataflowHook(gcp_conn_id="google_cloud_default") self.dataflow_hook.beam_hook = MagicMock() @@ -2268,10 +2264,6 @@ def func(mock_obj, return_value): class TestAsyncDataflowHook: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - AsyncDataflowHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") - @pytest.fixture def hook(self): return AsyncDataflowHook( diff --git a/providers/tests/google/cloud/hooks/test_dataform.py b/providers/tests/google/cloud/hooks/test_dataform.py index bedf91dadf6e..edf8211c0c07 100644 --- a/providers/tests/google/cloud/hooks/test_dataform.py +++ b/providers/tests/google/cloud/hooks/test_dataform.py @@ -60,10 +60,6 @@ class TestDataformHook: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - DataformHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") - def setup_method(self): with mock.patch( BASE_STRING.format("GoogleBaseHook.__init__"), diff --git a/providers/tests/google/cloud/hooks/test_datafusion.py b/providers/tests/google/cloud/hooks/test_datafusion.py index 271662f7bca5..cb82f932ae95 100644 --- a/providers/tests/google/cloud/hooks/test_datafusion.py +++ b/providers/tests/google/cloud/hooks/test_datafusion.py @@ -88,10 +88,6 @@ def session(): class TestDataFusionHook: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - DataFusionHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") - @staticmethod def mock_endpoint(get_conn_mock): return get_conn_mock.return_value.projects.return_value.locations.return_value.instances.return_value @@ -544,10 +540,6 @@ def test_cdap_program_id(self, pipeline_type, expected_program_id): class TestDataFusionHookAsynch: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - DataFusionAsyncHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") - @pytest.mark.asyncio @mock.patch(HOOK_STR.format("DataFusionAsyncHook._get_link")) async def test_async_get_pipeline_should_execute_successfully(self, mocked_link, hook_async): diff --git a/providers/tests/google/cloud/hooks/test_dataplex.py b/providers/tests/google/cloud/hooks/test_dataplex.py index 9221a0e09465..8f1f5d986661 100644 --- a/providers/tests/google/cloud/hooks/test_dataplex.py +++ b/providers/tests/google/cloud/hooks/test_dataplex.py @@ -18,7 +18,6 @@ from unittest import mock -import pytest from google.api_core.gapic_v1.method import DEFAULT from airflow.providers.google.cloud.operators.dataplex import DataplexHook @@ -54,10 +53,6 @@ class TestDataplexHook: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - DataplexHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") - def setup_method(self): with mock.patch( BASE_STRING.format("GoogleBaseHook.__init__"), diff --git a/providers/tests/google/cloud/hooks/test_dataproc.py b/providers/tests/google/cloud/hooks/test_dataproc.py index 0b44666e9165..88839dabb814 100644 --- a/providers/tests/google/cloud/hooks/test_dataproc.py +++ b/providers/tests/google/cloud/hooks/test_dataproc.py @@ -70,10 +70,6 @@ async def mock_awaitable(*args, **kwargs): class TestDataprocHook: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - DataprocHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") - def setup_method(self): with mock.patch(BASE_STRING.format("GoogleBaseHook.__init__"), new=mock_init): self.hook = DataprocHook(gcp_conn_id="test") @@ -550,10 +546,6 @@ def test_list_batches(self, mock_client): class TestDataprocAsyncHook: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - DataprocAsyncHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") - def setup_method(self): with mock.patch(BASE_STRING.format("GoogleBaseHook.__init__"), new=mock_init): self.hook = DataprocAsyncHook(gcp_conn_id="test") diff --git a/providers/tests/google/cloud/hooks/test_dataproc_metastore.py b/providers/tests/google/cloud/hooks/test_dataproc_metastore.py index 58a04d6bc835..600693fa0f3a 100644 --- a/providers/tests/google/cloud/hooks/test_dataproc_metastore.py +++ b/providers/tests/google/cloud/hooks/test_dataproc_metastore.py @@ -79,10 +79,6 @@ class TestDataprocMetastoreWithDefaultProjectIdHook: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - DataprocMetastoreHook(gcp_conn_id=TEST_GCP_CONN_ID, delegate_to="delegate_to") - def setup_method(self): with mock.patch( BASE_STRING.format("GoogleBaseHook.__init__"), new=mock_base_gcp_hook_default_project_id diff --git a/providers/tests/google/cloud/hooks/test_datastore.py b/providers/tests/google/cloud/hooks/test_datastore.py index d17a91d36b29..cfe42b35b77b 100644 --- a/providers/tests/google/cloud/hooks/test_datastore.py +++ b/providers/tests/google/cloud/hooks/test_datastore.py @@ -37,10 +37,6 @@ def mock_init( class TestDatastoreHook: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - DatastoreHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") - def setup_method(self): with patch( "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.__init__", new=mock_init diff --git a/providers/tests/google/cloud/hooks/test_dlp.py b/providers/tests/google/cloud/hooks/test_dlp.py index f0c63fcd2362..b623faadfebf 100644 --- a/providers/tests/google/cloud/hooks/test_dlp.py +++ b/providers/tests/google/cloud/hooks/test_dlp.py @@ -56,10 +56,6 @@ class TestCloudDLPHook: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - CloudDLPHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") - def setup_method(self): with mock.patch( "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.__init__", diff --git a/providers/tests/google/cloud/hooks/test_functions.py b/providers/tests/google/cloud/hooks/test_functions.py index 4e3c48795117..e52855caf84d 100644 --- a/providers/tests/google/cloud/hooks/test_functions.py +++ b/providers/tests/google/cloud/hooks/test_functions.py @@ -37,10 +37,6 @@ class TestFunctionHookNoDefaultProjectId: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - CloudFunctionsHook(api_version="v1", gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") - def setup_method(self): with mock.patch( "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.__init__", diff --git a/providers/tests/google/cloud/hooks/test_gcs.py b/providers/tests/google/cloud/hooks/test_gcs.py index 7c06990d08ca..48f8c4858117 100644 --- a/providers/tests/google/cloud/hooks/test_gcs.py +++ b/providers/tests/google/cloud/hooks/test_gcs.py @@ -172,10 +172,6 @@ def test_should_raise_exception_on_mutually_exclusive(self): class TestGCSHook: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - gcs.GCSHook(api_version="v1", gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") - def setup_method(self): with mock.patch( GCS_STRING.format("GoogleBaseHook.__init__"), @@ -1129,7 +1125,8 @@ def test_list_by_timespans(self, mock_service): class TestGCSHookUpload: def setup_method(self): - with mock.patch(BASE_STRING.format("GoogleBaseHook.__init__")): + with mock.patch(BASE_STRING.format("GoogleBaseHook.__init__")) as mock_init: + mock_init.return_value = None self.gcs_hook = gcs.GCSHook(gcp_conn_id="test") @mock.patch(GCS_STRING.format("GCSHook.get_conn")) diff --git a/providers/tests/google/cloud/hooks/test_gdm.py b/providers/tests/google/cloud/hooks/test_gdm.py index eabb0fb7501e..a6995f3c17f9 100644 --- a/providers/tests/google/cloud/hooks/test_gdm.py +++ b/providers/tests/google/cloud/hooks/test_gdm.py @@ -38,10 +38,6 @@ def mock_init( class TestDeploymentManagerHook: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - GoogleDeploymentManagerHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") - def setup_method(self): with mock.patch( "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.__init__", diff --git a/providers/tests/google/cloud/hooks/test_kms.py b/providers/tests/google/cloud/hooks/test_kms.py index e789cc43abef..0d3c33a880a3 100644 --- a/providers/tests/google/cloud/hooks/test_kms.py +++ b/providers/tests/google/cloud/hooks/test_kms.py @@ -21,7 +21,6 @@ from collections import namedtuple from unittest import mock -import pytest from google.api_core.gapic_v1.method import DEFAULT from airflow.providers.google.cloud.hooks.kms import CloudKMSHook @@ -57,10 +56,6 @@ def mock_init( class TestCloudKMSHook: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - CloudKMSHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") - def setup_method(self): with mock.patch( "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.__init__", diff --git a/providers/tests/google/cloud/hooks/test_kubernetes_engine.py b/providers/tests/google/cloud/hooks/test_kubernetes_engine.py index e535323f4d4c..9f563246ff44 100644 --- a/providers/tests/google/cloud/hooks/test_kubernetes_engine.py +++ b/providers/tests/google/cloud/hooks/test_kubernetes_engine.py @@ -164,10 +164,6 @@ @pytest.mark.db_test class TestGKEHookClient: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - GKEHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") - def setup_method(self): self.gke_hook = GKEHook(location=GKE_ZONE) @@ -422,7 +418,7 @@ def test_check_cluster_autoscaling_ability(self, cluster_obj, expected_result): assert result == expected_result -class TestGKEDeploymentHook: +class TestGKEKubernetesHookDeployments: def setup_method(self): with mock.patch( BASE_STRING.format("GoogleBaseHook.__init__"), new=mock_base_gcp_hook_default_project_id @@ -487,7 +483,7 @@ def test_check_kueue_deployment_raise_exception(self, gke_deployment_hook, caplo assert "Exception occurred while checking for Deployment status." in caplog.text -class TestGKEPodAsyncHook: +class TestGKEKubernetesAsyncHook: @staticmethod def make_mock_awaitable(mock_obj, result=None): f = Future() @@ -587,7 +583,7 @@ async def test_get_operation(self, mock_get_client, async_gke_hook, mock_async_g ) -class TestGKEPodHook: +class TestGKEKubernetesHookPod: def setup_method(self): with mock.patch( BASE_STRING.format("GoogleBaseHook.__init__"), new=mock_base_gcp_hook_default_project_id @@ -668,7 +664,7 @@ def test_disable_tcp_keepalive( assert isinstance(api_conn, kubernetes.client.api_client.ApiClient) -class TestGKEJobHook: +class TestGKEKubernetesHookJob: def setup_method(self): with mock.patch( BASE_STRING.format("GoogleBaseHook.__init__"), new=mock_base_gcp_hook_default_project_id @@ -713,7 +709,7 @@ def _get_credentials(self): return self.credentials -class TestGKECustomResourceHook: +class TestGKEKubernetesHookCustomResources: def setup_method(self): with mock.patch( BASE_STRING.format("GoogleBaseHook.__init__"), new=mock_base_gcp_hook_default_project_id diff --git a/providers/tests/google/cloud/hooks/test_life_sciences.py b/providers/tests/google/cloud/hooks/test_life_sciences.py index ca113ec648f5..990b9331daa2 100644 --- a/providers/tests/google/cloud/hooks/test_life_sciences.py +++ b/providers/tests/google/cloud/hooks/test_life_sciences.py @@ -50,10 +50,6 @@ class TestLifeSciencesHookWithPassedProjectId: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - LifeSciencesHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") - def setup_method(self): with mock.patch( "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.__init__", diff --git a/providers/tests/google/cloud/hooks/test_mlengine.py b/providers/tests/google/cloud/hooks/test_mlengine.py index 85ac30327d4c..d4ab1a365528 100644 --- a/providers/tests/google/cloud/hooks/test_mlengine.py +++ b/providers/tests/google/cloud/hooks/test_mlengine.py @@ -44,10 +44,6 @@ @pytest.mark.db_test class TestMLEngineHook: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - hook.MLEngineHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") - def setup_method(self): self.hook = hook.MLEngineHook() diff --git a/providers/tests/google/cloud/hooks/test_natural_language.py b/providers/tests/google/cloud/hooks/test_natural_language.py index a3228439b10e..6087fb3dc6a0 100644 --- a/providers/tests/google/cloud/hooks/test_natural_language.py +++ b/providers/tests/google/cloud/hooks/test_natural_language.py @@ -20,7 +20,6 @@ from typing import Any from unittest import mock -import pytest from google.api_core.gapic_v1.method import DEFAULT from google.cloud.language_v1 import Document @@ -37,10 +36,6 @@ class TestCloudNaturalLanguageHook: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - CloudNaturalLanguageHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") - def setup_method(self): with mock.patch( "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.__init__", diff --git a/providers/tests/google/cloud/hooks/test_os_login.py b/providers/tests/google/cloud/hooks/test_os_login.py index 48861f695469..e805cb34a924 100644 --- a/providers/tests/google/cloud/hooks/test_os_login.py +++ b/providers/tests/google/cloud/hooks/test_os_login.py @@ -48,10 +48,6 @@ class TestOSLoginHook: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - OSLoginHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") - def setup_method(self): with mock.patch( "airflow.providers.google.cloud.hooks.os_login.OSLoginHook.__init__", diff --git a/providers/tests/google/cloud/hooks/test_pubsub.py b/providers/tests/google/cloud/hooks/test_pubsub.py index 29c286b09192..7e5f99bfeafc 100644 --- a/providers/tests/google/cloud/hooks/test_pubsub.py +++ b/providers/tests/google/cloud/hooks/test_pubsub.py @@ -73,10 +73,6 @@ def _generate_messages(count) -> list[ReceivedMessage]: class TestPubSubHook: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - PubSubHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") - def setup_method(self): with mock.patch(BASE_STRING.format("GoogleBaseHook.__init__"), new=mock_init): self.pubsub_hook = PubSubHook(gcp_conn_id="test") diff --git a/providers/tests/google/cloud/hooks/test_secret_manager.py b/providers/tests/google/cloud/hooks/test_secret_manager.py index e758f76a9dcd..7a90240a8bf2 100644 --- a/providers/tests/google/cloud/hooks/test_secret_manager.py +++ b/providers/tests/google/cloud/hooks/test_secret_manager.py @@ -20,70 +20,17 @@ from unittest.mock import MagicMock, PropertyMock, patch import pytest -from google.api_core.exceptions import NotFound -from google.cloud.secretmanager_v1.types.service import AccessSecretVersionResponse -from airflow.exceptions import AirflowProviderDeprecationWarning -from airflow.providers.google.cloud.hooks.secret_manager import ( - GoogleCloudSecretManagerHook, - SecretsManagerHook, -) +from airflow.providers.google.cloud.hooks.secret_manager import GoogleCloudSecretManagerHook from airflow.providers.google.common.consts import CLIENT_INFO -from providers.tests.google.cloud.utils.base_gcp_mock import ( - GCP_PROJECT_ID_HOOK_UNIT_TEST, - mock_base_gcp_hook_default_project_id, -) +from providers.tests.google.cloud.utils.base_gcp_mock import GCP_PROJECT_ID_HOOK_UNIT_TEST BASE_PACKAGE = "airflow.providers.google.common.hooks.base_google." SECRETS_HOOK_PACKAGE = "airflow.providers.google.cloud.hooks.secret_manager." -INTERNAL_CLIENT_PACKAGE = "airflow.providers.google.cloud._internal_client.secret_manager_client" SECRET_ID = "test-secret-id" -class TestSecretsManagerHook: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - with pytest.warns(AirflowProviderDeprecationWarning): - SecretsManagerHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") - - @patch(INTERNAL_CLIENT_PACKAGE + "._SecretManagerClient.client", return_value=MagicMock()) - @patch( - SECRETS_HOOK_PACKAGE + "SecretsManagerHook.get_credentials_and_project_id", - return_value=(MagicMock(), GCP_PROJECT_ID_HOOK_UNIT_TEST), - ) - @patch(BASE_PACKAGE + "GoogleBaseHook.__init__", new=mock_base_gcp_hook_default_project_id) - def test_get_missing_key(self, mock_get_credentials, mock_client): - mock_client.secret_version_path.return_value = "full-path" - mock_client.access_secret_version.side_effect = NotFound("test-msg") - with pytest.warns(AirflowProviderDeprecationWarning): - secrets_manager_hook = SecretsManagerHook(gcp_conn_id="test") - mock_get_credentials.assert_called_once_with() - secret = secrets_manager_hook.get_secret(secret_id="secret") - mock_client.secret_version_path.assert_called_once_with("example-project", "secret", "latest") - mock_client.access_secret_version.assert_called_once_with(request={"name": "full-path"}) - assert secret is None - - @patch(INTERNAL_CLIENT_PACKAGE + "._SecretManagerClient.client", return_value=MagicMock()) - @patch( - SECRETS_HOOK_PACKAGE + "SecretsManagerHook.get_credentials_and_project_id", - return_value=(MagicMock(), GCP_PROJECT_ID_HOOK_UNIT_TEST), - ) - @patch(BASE_PACKAGE + "GoogleBaseHook.__init__", new=mock_base_gcp_hook_default_project_id) - def test_get_existing_key(self, mock_get_credentials, mock_client): - mock_client.secret_version_path.return_value = "full-path" - test_response = AccessSecretVersionResponse() - test_response.payload.data = b"result" - mock_client.access_secret_version.return_value = test_response - with pytest.warns(AirflowProviderDeprecationWarning): - secrets_manager_hook = SecretsManagerHook(gcp_conn_id="test") - mock_get_credentials.assert_called_once_with() - secret = secrets_manager_hook.get_secret(secret_id="secret") - mock_client.secret_version_path.assert_called_once_with("example-project", "secret", "latest") - mock_client.access_secret_version.assert_called_once_with(request={"name": "full-path"}) - assert "result" == secret - - class TestGoogleCloudSecretManagerHook: def setup_method(self, method): with patch(f"{BASE_PACKAGE}GoogleBaseHook.get_connection", return_value=MagicMock()): diff --git a/providers/tests/google/cloud/hooks/test_secret_manager_system.py b/providers/tests/google/cloud/hooks/test_secret_manager_system.py index 74bc08f9d6c7..f386c7941551 100644 --- a/providers/tests/google/cloud/hooks/test_secret_manager_system.py +++ b/providers/tests/google/cloud/hooks/test_secret_manager_system.py @@ -20,10 +20,7 @@ import pytest -from airflow.providers.google.cloud.hooks.secret_manager import SecretsManagerHook - -from providers.tests.google.cloud.utils.gcp_authenticator import GCP_SECRET_MANAGER_KEY -from tests_common.test_utils.gcp_system_helpers import GoogleSystemTest, provide_gcp_context +from tests_common.test_utils.gcp_system_helpers import GoogleSystemTest TEST_SECRET_ID = os.environ.get("GCP_SECRET_MANAGER_SECRET_ID", "test-secret") TEST_SECRET_VALUE = os.environ.get("GCP_SECRET_MANAGER_SECRET_VALUE", "test-secret-value") @@ -46,32 +43,3 @@ def helper_two_versions(): GoogleSystemTest.update_secret(TEST_SECRET_ID, TEST_SECRET_VALUE_UPDATED) yield GoogleSystemTest.delete_secret(TEST_SECRET_ID) - - -@pytest.mark.system("google.secret_manager") -@pytest.mark.credential_file(GCP_SECRET_MANAGER_KEY) -class TestSecretsManagerSystem(GoogleSystemTest): - @pytest.mark.usefixtures("helper_one_version") - @provide_gcp_context(GCP_SECRET_MANAGER_KEY) - def test_read_secret_from_secret_manager(self): - hook = SecretsManagerHook() - secret = hook.get_secret(secret_id=TEST_SECRET_ID) - assert TEST_SECRET_VALUE == secret - - @pytest.mark.usefixtures("helper_one_version") - @provide_gcp_context(GCP_SECRET_MANAGER_KEY) - def test_read_missing_secret_from_secret_manager(self): - hook = SecretsManagerHook() - secret = hook.get_secret(secret_id=TEST_MISSING_SECRET_ID) - assert secret is None - - @pytest.mark.usefixtures("helper_two_versions") - @provide_gcp_context(GCP_SECRET_MANAGER_KEY) - def test_read_secret_different_versions_from_secret_manager(self): - hook = SecretsManagerHook() - secret = hook.get_secret(secret_id=TEST_SECRET_ID) - assert TEST_SECRET_VALUE_UPDATED == secret - secret = hook.get_secret(secret_id=TEST_SECRET_ID, secret_version="1") - assert TEST_SECRET_VALUE == secret - secret = hook.get_secret(secret_id=TEST_SECRET_ID, secret_version="2") - assert TEST_SECRET_VALUE_UPDATED == secret diff --git a/providers/tests/google/cloud/hooks/test_spanner.py b/providers/tests/google/cloud/hooks/test_spanner.py index 3f6af9dcdcaf..ceeb85930cc5 100644 --- a/providers/tests/google/cloud/hooks/test_spanner.py +++ b/providers/tests/google/cloud/hooks/test_spanner.py @@ -20,7 +20,6 @@ from unittest import mock from unittest.mock import MagicMock, PropertyMock -import pytest import sqlalchemy from airflow.providers.google.cloud.hooks.spanner import SpannerHook @@ -40,10 +39,6 @@ class TestGcpSpannerHookDefaultProjectId: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - SpannerHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") - def setup_method(self): with mock.patch( "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.__init__", diff --git a/providers/tests/google/cloud/hooks/test_speech_to_text.py b/providers/tests/google/cloud/hooks/test_speech_to_text.py index 97ef56f790b4..3b0063106c82 100644 --- a/providers/tests/google/cloud/hooks/test_speech_to_text.py +++ b/providers/tests/google/cloud/hooks/test_speech_to_text.py @@ -19,7 +19,6 @@ from unittest.mock import patch -import pytest from google.api_core.gapic_v1.method import DEFAULT from google.cloud.speech_v1.types import RecognitionAudio, RecognitionConfig @@ -34,10 +33,6 @@ class TestCloudSpeechToTextHook: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - CloudSpeechToTextHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") - def setup_method(self): with patch( "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.__init__", diff --git a/providers/tests/google/cloud/hooks/test_stackdriver.py b/providers/tests/google/cloud/hooks/test_stackdriver.py index fc30c2312e9b..33c6f7bcd6dc 100644 --- a/providers/tests/google/cloud/hooks/test_stackdriver.py +++ b/providers/tests/google/cloud/hooks/test_stackdriver.py @@ -83,10 +83,6 @@ @pytest.mark.db_test class TestStackdriverHookMethods: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - stackdriver.StackdriverHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") - @mock.patch( "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.get_credentials_and_project_id", return_value=(CREDENTIALS, PROJECT_ID), diff --git a/providers/tests/google/cloud/hooks/test_tasks.py b/providers/tests/google/cloud/hooks/test_tasks.py index 71228ba7e717..aa59eddfa81b 100644 --- a/providers/tests/google/cloud/hooks/test_tasks.py +++ b/providers/tests/google/cloud/hooks/test_tasks.py @@ -20,7 +20,6 @@ from typing import Any from unittest import mock -import pytest from google.api_core.gapic_v1.method import DEFAULT from google.cloud.tasks_v2.types import Queue, Task @@ -49,10 +48,6 @@ class Obj: class TestCloudTasksHook: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - CloudTasksHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") - def setup_method(self): with mock.patch( "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.__init__", diff --git a/providers/tests/google/cloud/hooks/test_text_to_speech.py b/providers/tests/google/cloud/hooks/test_text_to_speech.py index 4f3eb9068449..6c3c635cd8ec 100644 --- a/providers/tests/google/cloud/hooks/test_text_to_speech.py +++ b/providers/tests/google/cloud/hooks/test_text_to_speech.py @@ -19,7 +19,6 @@ from unittest.mock import patch -import pytest from google.api_core.gapic_v1.method import DEFAULT from google.cloud.texttospeech_v1.types import ( AudioConfig, @@ -38,10 +37,6 @@ class TestTextToSpeechHook: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - CloudTextToSpeechHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") - def setup_method(self): with patch( "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.__init__", diff --git a/providers/tests/google/cloud/hooks/test_translate.py b/providers/tests/google/cloud/hooks/test_translate.py index addc54d430e7..42429bc58dda 100644 --- a/providers/tests/google/cloud/hooks/test_translate.py +++ b/providers/tests/google/cloud/hooks/test_translate.py @@ -19,8 +19,6 @@ from unittest import mock -import pytest - from airflow.providers.google.cloud.hooks.translate import CloudTranslateHook from airflow.providers.google.common.consts import CLIENT_INFO @@ -30,10 +28,6 @@ class TestCloudTranslateHook: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - CloudTranslateHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") - def setup_method(self): with mock.patch( "airflow.providers.google.cloud.hooks.translate.CloudTranslateHook.__init__", diff --git a/providers/tests/google/cloud/hooks/test_video_intelligence.py b/providers/tests/google/cloud/hooks/test_video_intelligence.py index 17e047e2da95..a8fdd75f0a79 100644 --- a/providers/tests/google/cloud/hooks/test_video_intelligence.py +++ b/providers/tests/google/cloud/hooks/test_video_intelligence.py @@ -19,7 +19,6 @@ from unittest import mock -import pytest from google.api_core.gapic_v1.method import DEFAULT from google.cloud.videointelligence_v1 import Feature @@ -37,10 +36,6 @@ class TestCloudVideoIntelligenceHook: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - CloudVideoIntelligenceHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") - def setup_method(self): with mock.patch( "airflow.providers.google.cloud.hooks.video_intelligence.CloudVideoIntelligenceHook.__init__", diff --git a/providers/tests/google/cloud/hooks/test_vision.py b/providers/tests/google/cloud/hooks/test_vision.py index 6a414d59e1a1..339258165961 100644 --- a/providers/tests/google/cloud/hooks/test_vision.py +++ b/providers/tests/google/cloud/hooks/test_vision.py @@ -88,10 +88,6 @@ class TestGcpVisionHook: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - CloudVisionHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") - def setup_method(self): with mock.patch( "airflow.providers.google.cloud.hooks.vision.CloudVisionHook.__init__", diff --git a/providers/tests/google/cloud/hooks/test_workflows.py b/providers/tests/google/cloud/hooks/test_workflows.py index cb5267ee5129..fd2e5c57044a 100644 --- a/providers/tests/google/cloud/hooks/test_workflows.py +++ b/providers/tests/google/cloud/hooks/test_workflows.py @@ -18,8 +18,6 @@ from unittest import mock -import pytest - from airflow.providers.google.cloud.hooks.workflows import WorkflowsHook from airflow.providers.google.common.consts import CLIENT_INFO @@ -52,10 +50,6 @@ def mock_init(*args, **kwargs): class TestWorkflowsHook: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - WorkflowsHook(gcp_conn_id="GCP_CONN_ID", delegate_to="delegate_to") - def setup_method(self, _): with mock.patch(BASE_PATH.format("GoogleBaseHook.__init__"), new=mock_init): self.hook = WorkflowsHook(gcp_conn_id="test") diff --git a/providers/tests/google/cloud/hooks/vertex_ai/test_auto_ml.py b/providers/tests/google/cloud/hooks/vertex_ai/test_auto_ml.py index 6b64c966f54c..0d875b7a5b97 100644 --- a/providers/tests/google/cloud/hooks/vertex_ai/test_auto_ml.py +++ b/providers/tests/google/cloud/hooks/vertex_ai/test_auto_ml.py @@ -115,10 +115,6 @@ def test_list_training_pipelines(self, mock_client) -> None: class TestAutoMLWithoutDefaultProjectIdHook: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - AutoMLHook(gcp_conn_id=TEST_GCP_CONN_ID, delegate_to="delegate_to") - def setup_method(self): with mock.patch( BASE_STRING.format("GoogleBaseHook.__init__"), new=mock_base_gcp_hook_no_default_project_id diff --git a/providers/tests/google/cloud/hooks/vertex_ai/test_batch_prediction_job.py b/providers/tests/google/cloud/hooks/vertex_ai/test_batch_prediction_job.py index 15efe3e88bac..966543f89cfd 100644 --- a/providers/tests/google/cloud/hooks/vertex_ai/test_batch_prediction_job.py +++ b/providers/tests/google/cloud/hooks/vertex_ai/test_batch_prediction_job.py @@ -80,10 +80,6 @@ class TestBatchPredictionJobWithDefaultProjectIdHook: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - BatchPredictionJobHook(gcp_conn_id=TEST_GCP_CONN_ID, delegate_to="delegate_to") - def setup_method(self): with mock.patch( BASE_STRING.format("GoogleBaseHook.__init__"), new=mock_base_gcp_hook_default_project_id diff --git a/providers/tests/google/cloud/hooks/vertex_ai/test_custom_job.py b/providers/tests/google/cloud/hooks/vertex_ai/test_custom_job.py index a2cedce2a92a..6a7d6215a132 100644 --- a/providers/tests/google/cloud/hooks/vertex_ai/test_custom_job.py +++ b/providers/tests/google/cloud/hooks/vertex_ai/test_custom_job.py @@ -99,10 +99,6 @@ def test_custom_job_name(job_service_async_client): class TestCustomJobWithDefaultProjectIdHook: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - CustomJobHook(gcp_conn_id=TEST_GCP_CONN_ID, delegate_to="delegate_to") - def setup_method(self): with mock.patch( BASE_STRING.format("GoogleBaseHook.__init__"), new=mock_base_gcp_hook_default_project_id diff --git a/providers/tests/google/cloud/hooks/vertex_ai/test_dataset.py b/providers/tests/google/cloud/hooks/vertex_ai/test_dataset.py index d54eb48c2d68..cbc95363fe24 100644 --- a/providers/tests/google/cloud/hooks/vertex_ai/test_dataset.py +++ b/providers/tests/google/cloud/hooks/vertex_ai/test_dataset.py @@ -53,10 +53,6 @@ class TestVertexAIWithDefaultProjectIdHook: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - DatasetHook(gcp_conn_id=TEST_GCP_CONN_ID, delegate_to="delegate_to") - def setup_method(self): with mock.patch( BASE_STRING.format("GoogleBaseHook.__init__"), new=mock_base_gcp_hook_default_project_id diff --git a/providers/tests/google/cloud/hooks/vertex_ai/test_endpoint_service.py b/providers/tests/google/cloud/hooks/vertex_ai/test_endpoint_service.py index a284eae13a41..695d3f7936bd 100644 --- a/providers/tests/google/cloud/hooks/vertex_ai/test_endpoint_service.py +++ b/providers/tests/google/cloud/hooks/vertex_ai/test_endpoint_service.py @@ -49,10 +49,6 @@ class TestEndpointServiceWithDefaultProjectIdHook: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - EndpointServiceHook(gcp_conn_id=TEST_GCP_CONN_ID, delegate_to="delegate_to") - def setup_method(self): with mock.patch( BASE_STRING.format("GoogleBaseHook.__init__"), new=mock_base_gcp_hook_default_project_id diff --git a/providers/tests/google/cloud/hooks/vertex_ai/test_hyperparameter_tuning_job.py b/providers/tests/google/cloud/hooks/vertex_ai/test_hyperparameter_tuning_job.py index 05fbf9c12344..19f3c2e1f9ce 100644 --- a/providers/tests/google/cloud/hooks/vertex_ai/test_hyperparameter_tuning_job.py +++ b/providers/tests/google/cloud/hooks/vertex_ai/test_hyperparameter_tuning_job.py @@ -131,10 +131,6 @@ class TestHyperparameterTuningJobWithDefaultProjectIdHook: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - HyperparameterTuningJobHook(gcp_conn_id=TEST_GCP_CONN_ID, delegate_to="delegate_to") - def setup_method(self): with mock.patch( BASE_STRING.format("GoogleBaseHook.__init__"), new=mock_base_gcp_hook_default_project_id diff --git a/providers/tests/google/cloud/hooks/vertex_ai/test_model_service.py b/providers/tests/google/cloud/hooks/vertex_ai/test_model_service.py index ea337576e0cb..502d68660f1b 100644 --- a/providers/tests/google/cloud/hooks/vertex_ai/test_model_service.py +++ b/providers/tests/google/cloud/hooks/vertex_ai/test_model_service.py @@ -47,10 +47,6 @@ class TestModelServiceWithDefaultProjectIdHook: - def test_delegate_to_runtime_error(self): - with pytest.raises(RuntimeError): - ModelServiceHook(gcp_conn_id=TEST_GCP_CONN_ID, delegate_to="delegate_to") - def setup_method(self): with mock.patch( BASE_STRING.format("GoogleBaseHook.__init__"), new=mock_base_gcp_hook_default_project_id diff --git a/providers/tests/google/cloud/operators/test_bigquery.py b/providers/tests/google/cloud/operators/test_bigquery.py index 4269d02377b2..1088f8489b4a 100644 --- a/providers/tests/google/cloud/operators/test_bigquery.py +++ b/providers/tests/google/cloud/operators/test_bigquery.py @@ -30,7 +30,6 @@ from airflow.exceptions import ( AirflowException, - AirflowProviderDeprecationWarning, AirflowSkipException, AirflowTaskTimeout, TaskDeferred, @@ -44,20 +43,16 @@ from airflow.providers.google.cloud.operators.bigquery import ( BigQueryCheckOperator, BigQueryColumnCheckOperator, - BigQueryConsoleIndexableLink, - BigQueryConsoleLink, BigQueryCreateEmptyDatasetOperator, BigQueryCreateEmptyTableOperator, BigQueryCreateExternalTableOperator, BigQueryDeleteDatasetOperator, BigQueryDeleteTableOperator, - BigQueryExecuteQueryOperator, BigQueryGetDataOperator, BigQueryGetDatasetOperator, BigQueryGetDatasetTablesOperator, BigQueryInsertJobOperator, BigQueryIntervalCheckOperator, - BigQueryPatchDatasetOperator, BigQueryTableCheckOperator, BigQueryUpdateDatasetOperator, BigQueryUpdateTableOperator, @@ -72,16 +67,8 @@ BigQueryIntervalCheckTrigger, BigQueryValueCheckTrigger, ) -from airflow.serialization.serialized_objects import SerializedDAG from airflow.utils.timezone import datetime -from tests_common.test_utils.db import ( - clear_db_dags, - clear_db_runs, - clear_db_serialized_dags, - clear_db_xcom, -) - pytestmark = pytest.mark.db_test @@ -488,24 +475,6 @@ def test_execute(self, mock_hook): ) -class TestBigQueryPatchDatasetOperator: - @mock.patch("airflow.providers.google.cloud.operators.bigquery.BigQueryHook") - def test_execute(self, mock_hook): - dataset_resource = {"friendlyName": "Test DS"} - with pytest.warns(AirflowProviderDeprecationWarning): - operator = BigQueryPatchDatasetOperator( - dataset_resource=dataset_resource, - task_id=TASK_ID, - dataset_id=TEST_DATASET, - project_id=TEST_GCP_PROJECT_ID, - ) - - operator.execute(None) - mock_hook.return_value.patch_dataset.assert_called_once_with( - dataset_resource=dataset_resource, dataset_id=TEST_DATASET, project_id=TEST_GCP_PROJECT_ID - ) - - class TestBigQueryUpdateDatasetOperator: @mock.patch("airflow.providers.google.cloud.operators.bigquery.BigQueryHook") def test_execute(self, mock_hook): @@ -526,373 +495,6 @@ def test_execute(self, mock_hook): ) -@pytest.mark.db_test -class TestBigQueryOperator: - def teardown_method(self): - clear_db_xcom() - clear_db_runs() - clear_db_serialized_dags() - clear_db_dags() - - @mock.patch("airflow.providers.google.cloud.operators.bigquery.BigQueryHook") - def test_execute(self, mock_hook): - encryption_configuration = {"key": "kk"} - - with pytest.warns(AirflowProviderDeprecationWarning): - operator = BigQueryExecuteQueryOperator( - task_id=TASK_ID, - sql="Select * from test_table", - destination_dataset_table=None, - write_disposition="WRITE_EMPTY", - allow_large_results=False, - flatten_results=None, - gcp_conn_id="google_cloud_default", - udf_config=None, - use_legacy_sql=True, - maximum_billing_tier=None, - maximum_bytes_billed=None, - create_disposition="CREATE_IF_NEEDED", - schema_update_options=(), - query_params=None, - labels=None, - priority="INTERACTIVE", - time_partitioning=None, - api_resource_configs=None, - cluster_fields=None, - encryption_configuration=encryption_configuration, - impersonation_chain=["service-account@myproject.iam.gserviceaccount.com"], - impersonation_scopes=[ - "https://www.googleapis.com/auth/cloud-platform", - "https://www.googleapis.com/auth/drive", - ], - ) - - operator.execute(MagicMock()) - mock_hook.assert_called_with( - gcp_conn_id="google_cloud_default", - use_legacy_sql=True, - location=None, - impersonation_chain=["service-account@myproject.iam.gserviceaccount.com"], - impersonation_scopes=[ - "https://www.googleapis.com/auth/cloud-platform", - "https://www.googleapis.com/auth/drive", - ], - ) - mock_hook.return_value.run_query.assert_called_once_with( - sql="Select * from test_table", - destination_dataset_table=None, - write_disposition="WRITE_EMPTY", - allow_large_results=False, - flatten_results=None, - udf_config=None, - maximum_billing_tier=None, - maximum_bytes_billed=None, - create_disposition="CREATE_IF_NEEDED", - schema_update_options=(), - query_params=None, - labels=None, - priority="INTERACTIVE", - time_partitioning=None, - api_resource_configs=None, - cluster_fields=None, - encryption_configuration=encryption_configuration, - ) - - @mock.patch("airflow.providers.google.cloud.operators.bigquery.BigQueryHook") - def test_execute_list(self, mock_hook): - with pytest.warns(AirflowProviderDeprecationWarning): - operator = BigQueryExecuteQueryOperator( - task_id=TASK_ID, - sql=[ - "Select * from test_table", - "Select * from other_test_table", - ], - destination_dataset_table=None, - write_disposition="WRITE_EMPTY", - allow_large_results=False, - flatten_results=None, - gcp_conn_id="google_cloud_default", - udf_config=None, - use_legacy_sql=True, - maximum_billing_tier=None, - maximum_bytes_billed=None, - create_disposition="CREATE_IF_NEEDED", - schema_update_options=(), - query_params=None, - labels=None, - priority="INTERACTIVE", - time_partitioning=None, - api_resource_configs=None, - cluster_fields=None, - encryption_configuration=None, - ) - - operator.execute(MagicMock()) - mock_hook.return_value.run_query.assert_has_calls( - [ - mock.call( - sql="Select * from test_table", - destination_dataset_table=None, - write_disposition="WRITE_EMPTY", - allow_large_results=False, - flatten_results=None, - udf_config=None, - maximum_billing_tier=None, - maximum_bytes_billed=None, - create_disposition="CREATE_IF_NEEDED", - schema_update_options=(), - query_params=None, - labels=None, - priority="INTERACTIVE", - time_partitioning=None, - api_resource_configs=None, - cluster_fields=None, - encryption_configuration=None, - ), - mock.call( - sql="Select * from other_test_table", - destination_dataset_table=None, - write_disposition="WRITE_EMPTY", - allow_large_results=False, - flatten_results=None, - udf_config=None, - maximum_billing_tier=None, - maximum_bytes_billed=None, - create_disposition="CREATE_IF_NEEDED", - schema_update_options=(), - query_params=None, - labels=None, - priority="INTERACTIVE", - time_partitioning=None, - api_resource_configs=None, - cluster_fields=None, - encryption_configuration=None, - ), - ] - ) - - @mock.patch("airflow.providers.google.cloud.operators.bigquery.BigQueryHook") - def test_execute_bad_type(self, mock_hook): - with pytest.warns(AirflowProviderDeprecationWarning): - operator = BigQueryExecuteQueryOperator( - task_id=TASK_ID, - sql=1, - destination_dataset_table=None, - write_disposition="WRITE_EMPTY", - allow_large_results=False, - flatten_results=None, - gcp_conn_id="google_cloud_default", - udf_config=None, - use_legacy_sql=True, - maximum_billing_tier=None, - maximum_bytes_billed=None, - create_disposition="CREATE_IF_NEEDED", - schema_update_options=(), - query_params=None, - labels=None, - priority="INTERACTIVE", - time_partitioning=None, - api_resource_configs=None, - cluster_fields=None, - ) - - with pytest.raises(AirflowException): - operator.execute(MagicMock()) - - @mock.patch("airflow.providers.google.cloud.operators.bigquery.BigQueryHook") - def test_bigquery_operator_defaults(self, mock_hook, create_task_instance_of_operator, session): - with pytest.warns(AirflowProviderDeprecationWarning): - ti = create_task_instance_of_operator( - BigQueryExecuteQueryOperator, - dag_id=TEST_DAG_ID, - task_id=TASK_ID, - sql="Select * from test_table", - schema_update_options=None, - ) - session.add(ti) - session.commit() - operator = ti.task - - operator.execute(MagicMock()) - mock_hook.return_value.run_query.assert_called_once_with( - sql="Select * from test_table", - destination_dataset_table=None, - write_disposition="WRITE_EMPTY", - allow_large_results=False, - flatten_results=None, - udf_config=None, - maximum_billing_tier=None, - maximum_bytes_billed=None, - create_disposition="CREATE_IF_NEEDED", - schema_update_options=None, - query_params=None, - labels=None, - priority="INTERACTIVE", - time_partitioning=None, - api_resource_configs=None, - cluster_fields=None, - encryption_configuration=None, - ) - assert isinstance(operator.sql, str) - ti.render_templates() - assert isinstance(ti.task.sql, str) - - @pytest.mark.need_serialized_dag - def test_bigquery_operator_extra_serialized_field_when_single_query( - self, - dag_maker, - create_task_instance_of_operator, - session, - ): - with pytest.warns(AirflowProviderDeprecationWarning): - ti = create_task_instance_of_operator( - BigQueryExecuteQueryOperator, - dag_id=TEST_DAG_ID, - execution_date=DEFAULT_DATE, - task_id=TASK_ID, - sql="SELECT * FROM test_table", - ) - session.add(ti) - session.commit() - serialized_dag = dag_maker.get_serialized_data() - deserialized_dag = SerializedDAG.deserialize_dag(serialized_dag["dag"]) - assert hasattr(deserialized_dag.tasks[0], "sql") - - dag = SerializedDAG.from_dict(serialized_dag) - simple_task = dag.task_dict[TASK_ID] - assert getattr(simple_task, "sql") == "SELECT * FROM test_table" - - ######################################################### - # Verify Operator Links work with Serialized Operator - ######################################################### - deserialized_dag = SerializedDAG.deserialize_dag(serialized_dag["dag"]) - assert deserialized_dag.tasks[0].operator_extra_links[0].name == "BigQuery Console" - - # Check DeSerialized version of operator link - assert isinstance(next(iter(simple_task.operator_extra_links)), BigQueryConsoleLink) - - ti.xcom_push("job_id_path", TEST_FULL_JOB_ID) - - url = simple_task.get_extra_links(ti, BigQueryConsoleLink.name) - assert url == f"https://console.cloud.google.com/bigquery?j={TEST_FULL_JOB_ID}" - - @pytest.mark.need_serialized_dag - def test_bigquery_operator_extra_serialized_field_when_multiple_queries( - self, - dag_maker, - create_task_instance_of_operator, - ): - with pytest.warns(AirflowProviderDeprecationWarning): - ti = create_task_instance_of_operator( - BigQueryExecuteQueryOperator, - dag_id=TEST_DAG_ID, - execution_date=DEFAULT_DATE, - task_id=TASK_ID, - sql=["SELECT * FROM test_table", "SELECT * FROM test_table2"], - ) - serialized_dag = dag_maker.get_serialized_data() - deserialized_dag = SerializedDAG.deserialize_dag(serialized_dag["dag"]) - assert hasattr(deserialized_dag.tasks[0], "sql") - - dag = SerializedDAG.from_dict(serialized_dag) - simple_task = dag.task_dict[TASK_ID] - assert getattr(simple_task, "sql") == ["SELECT * FROM test_table", "SELECT * FROM test_table2"] - - ######################################################### - # Verify Operator Links work with Serialized Operator - ######################################################### - deserialized_dag = SerializedDAG.deserialize_dag(serialized_dag["dag"]) - operator_extra_links = deserialized_dag.tasks[0].operator_extra_links - assert operator_extra_links[0].name == "BigQuery Console #1" - assert operator_extra_links[1].name == "BigQuery Console #2" - - # Check DeSerialized version of operator link - assert isinstance(next(iter(simple_task.operator_extra_links)), BigQueryConsoleIndexableLink) - - ti.xcom_push(key="job_id_path", value=[TEST_FULL_JOB_ID, TEST_FULL_JOB_ID_2]) - - assert {"BigQuery Console #1", "BigQuery Console #2"} == simple_task.operator_extra_link_dict.keys() - - assert ( - f"https://console.cloud.google.com/bigquery?j={TEST_FULL_JOB_ID}" - == simple_task.get_extra_links(ti, "BigQuery Console #1") - ) - - assert ( - f"https://console.cloud.google.com/bigquery?j={TEST_FULL_JOB_ID_2}" - == simple_task.get_extra_links(ti, "BigQuery Console #2") - ) - - @mock.patch("airflow.providers.google.cloud.operators.bigquery.BigQueryHook") - def test_bigquery_operator_extra_link_when_missing_job_id( - self, mock_hook, create_task_instance_of_operator - ): - with pytest.warns(AirflowProviderDeprecationWarning): - ti = create_task_instance_of_operator( - BigQueryExecuteQueryOperator, - dag_id=TEST_DAG_ID, - task_id=TASK_ID, - sql="SELECT * FROM test_table", - ) - bigquery_task = ti.task - - assert "" == bigquery_task.get_extra_links(ti, BigQueryConsoleLink.name) - - @mock.patch("airflow.providers.google.cloud.operators.bigquery.BigQueryHook") - def test_bigquery_operator_extra_link_when_single_query( - self, - mock_hook, - create_task_instance_of_operator, - session, - ): - with pytest.warns(AirflowProviderDeprecationWarning): - ti = create_task_instance_of_operator( - BigQueryExecuteQueryOperator, - dag_id=TEST_DAG_ID, - execution_date=DEFAULT_DATE, - task_id=TASK_ID, - sql="SELECT * FROM test_table", - ) - bigquery_task = ti.task - session.add(ti) - session.commit() - ti.xcom_push(key="job_id_path", value=TEST_FULL_JOB_ID) - - assert ( - f"https://console.cloud.google.com/bigquery?j={TEST_FULL_JOB_ID}" - == bigquery_task.get_extra_links(ti, BigQueryConsoleLink.name) - ) - - @mock.patch("airflow.providers.google.cloud.operators.bigquery.BigQueryHook") - def test_bigquery_operator_extra_link_when_multiple_query( - self, mock_hook, create_task_instance_of_operator, session - ): - with pytest.warns(AirflowProviderDeprecationWarning): - ti = create_task_instance_of_operator( - BigQueryExecuteQueryOperator, - dag_id=TEST_DAG_ID, - execution_date=DEFAULT_DATE, - task_id=TASK_ID, - sql=["SELECT * FROM test_table", "SELECT * FROM test_table2"], - ) - bigquery_task = ti.task - session.add(ti) - session.commit() - ti.xcom_push(key="job_id_path", value=[TEST_FULL_JOB_ID, TEST_FULL_JOB_ID_2]) - - assert {"BigQuery Console #1", "BigQuery Console #2"} == bigquery_task.operator_extra_link_dict.keys() - - assert ( - f"https://console.cloud.google.com/bigquery?j={TEST_FULL_JOB_ID}" - == bigquery_task.get_extra_links(ti, "BigQuery Console #1") - ) - - assert ( - f"https://console.cloud.google.com/bigquery?j={TEST_FULL_JOB_ID_2}" - == bigquery_task.get_extra_links(ti, "BigQuery Console #2") - ) - - class TestBigQueryGetDataOperator: @pytest.mark.parametrize("as_dict", [True, False]) @mock.patch("airflow.providers.google.cloud.operators.bigquery.BigQueryHook") diff --git a/providers/tests/google/cloud/operators/test_dataflow.py b/providers/tests/google/cloud/operators/test_dataflow.py index bc23d84d2285..83b33eaccf00 100644 --- a/providers/tests/google/cloud/operators/test_dataflow.py +++ b/providers/tests/google/cloud/operators/test_dataflow.py @@ -17,8 +17,6 @@ # under the License. from __future__ import annotations -import copy -import warnings from copy import deepcopy from unittest import mock @@ -26,17 +24,13 @@ import pytest from googleapiclient.errors import HttpError -import airflow from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning from airflow.providers.google.cloud.hooks.dataflow import ( DEFAULT_DATAFLOW_LOCATION, DataflowJobStatus, ) from airflow.providers.google.cloud.operators.dataflow import ( - CheckJobRunning, - DataflowCreateJavaJobOperator, DataflowCreatePipelineOperator, - DataflowCreatePythonJobOperator, DataflowDeletePipelineOperator, DataflowRunPipelineOperator, DataflowStartFlexTemplateOperator, @@ -134,361 +128,6 @@ } -class TestDataflowCreatePythonJobOperator: - def setup_method(self): - with warnings.catch_warnings(): - warnings.simplefilter("ignore", AirflowProviderDeprecationWarning) - self.dataflow = DataflowCreatePythonJobOperator( - task_id=TASK_ID, - py_file=PY_FILE, - job_name=JOB_NAME, - py_options=PY_OPTIONS, - dataflow_default_options=DEFAULT_OPTIONS_PYTHON, - options=ADDITIONAL_OPTIONS, - poll_sleep=POLL_SLEEP, - location=TEST_LOCATION, - ) - self.expected_airflow_version = "v" + airflow.version.version.replace(".", "-").replace("+", "-") - - def test_init(self): - """Test DataflowCreatePythonJobOperator instance is properly initialized.""" - assert self.dataflow.task_id == TASK_ID - assert self.dataflow.job_name == JOB_NAME - assert self.dataflow.py_file == PY_FILE - assert self.dataflow.py_options == PY_OPTIONS - assert self.dataflow.py_interpreter == PY_INTERPRETER - assert self.dataflow.poll_sleep == POLL_SLEEP - assert self.dataflow.dataflow_default_options == DEFAULT_OPTIONS_PYTHON - assert self.dataflow.options == EXPECTED_ADDITIONAL_OPTIONS - - @mock.patch( - "airflow.providers.google.cloud.operators.dataflow.process_line_and_extract_dataflow_job_id_callback" - ) - @mock.patch("airflow.providers.google.cloud.operators.dataflow.BeamHook") - @mock.patch("airflow.providers.google.cloud.operators.dataflow.DataflowHook") - @mock.patch("airflow.providers.google.cloud.operators.dataflow.GCSHook") - def test_exec(self, gcs_hook, dataflow_hook_mock, beam_hook_mock, mock_callback_on_job_id): - """Test DataflowHook is created and the right args are passed to - start_python_workflow. - - """ - start_python_mock = beam_hook_mock.return_value.start_python_pipeline - provide_gcloud_mock = dataflow_hook_mock.return_value.provide_authorized_gcloud - gcs_provide_file = gcs_hook.return_value.provide_file - job_name = dataflow_hook_mock.return_value.build_dataflow_job_name.return_value - self.dataflow.execute(None) - beam_hook_mock.assert_called_once_with(runner="DataflowRunner") - assert self.dataflow.py_file.startswith("/tmp/dataflow") - gcs_provide_file.assert_called_once_with(object_url=PY_FILE) - mock_callback_on_job_id.assert_called_once_with(on_new_job_id_callback=mock.ANY) - dataflow_hook_mock.assert_called_once_with( - gcp_conn_id="google_cloud_default", - poll_sleep=POLL_SLEEP, - impersonation_chain=None, - drain_pipeline=False, - cancel_timeout=mock.ANY, - wait_until_finished=None, - ) - expected_options = { - "project": dataflow_hook_mock.return_value.project_id, - "staging_location": "gs://test/staging", - "job_name": job_name, - "region": TEST_LOCATION, - "output": "gs://test/output", - "labels": {"foo": "bar", "airflow-version": self.expected_airflow_version}, - } - start_python_mock.assert_called_once_with( - variables=expected_options, - py_file=gcs_provide_file.return_value.__enter__.return_value.name, - py_options=PY_OPTIONS, - py_interpreter=PY_INTERPRETER, - py_requirements=None, - py_system_site_packages=False, - process_line_callback=mock_callback_on_job_id.return_value, - ) - dataflow_hook_mock.return_value.wait_for_done.assert_called_once_with( - job_id=mock.ANY, - job_name=job_name, - location=TEST_LOCATION, - multiple_jobs=False, - ) - assert self.dataflow.py_file.startswith("/tmp/dataflow") - provide_gcloud_mock.assert_called_once_with() - - -class TestDataflowCreateJavaJobOperator: - def setup_method(self): - with warnings.catch_warnings(): - warnings.simplefilter("ignore", AirflowProviderDeprecationWarning) - self.dataflow = DataflowCreateJavaJobOperator( - task_id=TASK_ID, - jar=JAR_FILE, - job_name=JOB_NAME, - job_class=JOB_CLASS, - dataflow_default_options=DEFAULT_OPTIONS_JAVA, - options=ADDITIONAL_OPTIONS, - poll_sleep=POLL_SLEEP, - location=TEST_LOCATION, - ) - self.expected_airflow_version = "v" + airflow.version.version.replace(".", "-").replace("+", "-") - - def test_init(self): - """Test DataflowCreateJavaJobOperator instance is properly initialized.""" - assert self.dataflow.task_id == TASK_ID - assert self.dataflow.job_name == JOB_NAME - assert self.dataflow.poll_sleep == POLL_SLEEP - assert self.dataflow.dataflow_default_options == DEFAULT_OPTIONS_JAVA - assert self.dataflow.job_class == JOB_CLASS - assert self.dataflow.jar == JAR_FILE - assert self.dataflow.options == EXPECTED_ADDITIONAL_OPTIONS - assert self.dataflow.check_if_running == CheckJobRunning.WaitForRun - - @mock.patch( - "airflow.providers.google.cloud.operators.dataflow.process_line_and_extract_dataflow_job_id_callback" - ) - @mock.patch("airflow.providers.google.cloud.operators.dataflow.BeamHook") - @mock.patch("airflow.providers.google.cloud.operators.dataflow.DataflowHook") - @mock.patch("airflow.providers.google.cloud.operators.dataflow.GCSHook") - def test_exec(self, gcs_hook, dataflow_hook_mock, beam_hook_mock, mock_callback_on_job_id): - """Test DataflowHook is created and the right args are passed to - start_java_workflow. - - """ - start_java_mock = beam_hook_mock.return_value.start_java_pipeline - gcs_provide_file = gcs_hook.return_value.provide_file - job_name = dataflow_hook_mock.return_value.build_dataflow_job_name.return_value - provide_gcloud_mock = dataflow_hook_mock.return_value.provide_authorized_gcloud - self.dataflow.check_if_running = CheckJobRunning.IgnoreJob - - self.dataflow.execute(None) - - mock_callback_on_job_id.assert_called_once_with(on_new_job_id_callback=mock.ANY) - gcs_provide_file.assert_called_once_with(object_url=JAR_FILE) - expected_variables = { - "project": dataflow_hook_mock.return_value.project_id, - "stagingLocation": "gs://test/staging", - "jobName": job_name, - "region": TEST_LOCATION, - "output": "gs://test/output", - "labels": {"foo": "bar", "airflow-version": self.expected_airflow_version}, - } - - start_java_mock.assert_called_once_with( - variables=expected_variables, - jar=gcs_provide_file.return_value.__enter__.return_value.name, - job_class=JOB_CLASS, - process_line_callback=mock_callback_on_job_id.return_value, - ) - dataflow_hook_mock.return_value.wait_for_done.assert_called_once_with( - job_id=mock.ANY, - job_name=job_name, - location=TEST_LOCATION, - multiple_jobs=False, - ) - - provide_gcloud_mock.assert_called_once_with() - - @mock.patch("airflow.providers.google.cloud.operators.dataflow.BeamHook") - @mock.patch("airflow.providers.google.cloud.operators.dataflow.DataflowHook") - @mock.patch("airflow.providers.google.cloud.operators.dataflow.GCSHook") - def test_check_job_running_exec(self, gcs_hook, dataflow_mock, beam_hook_mock): - """Test DataflowHook is created and the right args are passed to - start_java_workflow. - - """ - dataflow_running = dataflow_mock.return_value.is_job_dataflow_running - dataflow_running.return_value = True - start_java_hook = beam_hook_mock.return_value.start_java_pipeline - gcs_provide_file = gcs_hook.return_value.provide_file - self.dataflow.check_if_running = True - - self.dataflow.execute(None) - - assert dataflow_mock.called - start_java_hook.assert_not_called() - gcs_provide_file.assert_called_once() - variables = { - "project": dataflow_mock.return_value.project_id, - "stagingLocation": "gs://test/staging", - "jobName": JOB_NAME, - "region": TEST_LOCATION, - "output": "gs://test/output", - "labels": {"foo": "bar", "airflow-version": self.expected_airflow_version}, - } - dataflow_running.assert_called_once_with(name=JOB_NAME, variables=variables, location=TEST_LOCATION) - - @mock.patch( - "airflow.providers.google.cloud.operators.dataflow.process_line_and_extract_dataflow_job_id_callback" - ) - @mock.patch("airflow.providers.google.cloud.operators.dataflow.BeamHook") - @mock.patch("airflow.providers.google.cloud.operators.dataflow.DataflowHook") - @mock.patch("airflow.providers.google.cloud.operators.dataflow.GCSHook") - def test_check_job_not_running_exec( - self, gcs_hook, dataflow_hook_mock, beam_hook_mock, mock_callback_on_job_id - ): - """Test DataflowHook is created and the right args are passed to - start_java_workflow with option to check if job is running - """ - is_job_dataflow_running_variables = None - - def set_is_job_dataflow_running_variables(*args, **kwargs): - nonlocal is_job_dataflow_running_variables - is_job_dataflow_running_variables = copy.deepcopy(kwargs.get("variables")) - - dataflow_running = dataflow_hook_mock.return_value.is_job_dataflow_running - dataflow_running.side_effect = set_is_job_dataflow_running_variables - dataflow_running.return_value = False - start_java_mock = beam_hook_mock.return_value.start_java_pipeline - gcs_provide_file = gcs_hook.return_value.provide_file - self.dataflow.check_if_running = True - - self.dataflow.execute(None) - - mock_callback_on_job_id.assert_called_once_with(on_new_job_id_callback=mock.ANY) - gcs_provide_file.assert_called_once_with(object_url=JAR_FILE) - expected_variables = { - "project": dataflow_hook_mock.return_value.project_id, - "stagingLocation": "gs://test/staging", - "jobName": JOB_NAME, - "region": TEST_LOCATION, - "output": "gs://test/output", - "labels": {"foo": "bar", "airflow-version": self.expected_airflow_version}, - } - assert expected_variables == is_job_dataflow_running_variables - job_name = dataflow_hook_mock.return_value.build_dataflow_job_name.return_value - expected_variables["jobName"] = job_name - start_java_mock.assert_called_once_with( - variables=expected_variables, - jar=gcs_provide_file.return_value.__enter__.return_value.name, - job_class=JOB_CLASS, - process_line_callback=mock_callback_on_job_id.return_value, - ) - dataflow_hook_mock.return_value.wait_for_done.assert_called_once_with( - job_id=mock.ANY, - job_name=job_name, - location=TEST_LOCATION, - multiple_jobs=False, - ) - - @mock.patch( - "airflow.providers.google.cloud.operators.dataflow.process_line_and_extract_dataflow_job_id_callback" - ) - @mock.patch("airflow.providers.google.cloud.operators.dataflow.BeamHook") - @mock.patch("airflow.providers.google.cloud.operators.dataflow.DataflowHook") - @mock.patch("airflow.providers.google.cloud.operators.dataflow.GCSHook") - def test_check_multiple_job_exec( - self, gcs_hook, dataflow_hook_mock, beam_hook_mock, mock_callback_on_job_id - ): - """Test DataflowHook is created and the right args are passed to - start_java_workflow with option to check if job is running - """ - is_job_dataflow_running_variables = None - - def set_is_job_dataflow_running_variables(*args, **kwargs): - nonlocal is_job_dataflow_running_variables - is_job_dataflow_running_variables = copy.deepcopy(kwargs.get("variables")) - - dataflow_running = dataflow_hook_mock.return_value.is_job_dataflow_running - dataflow_running.side_effect = set_is_job_dataflow_running_variables - dataflow_running.return_value = False - start_java_mock = beam_hook_mock.return_value.start_java_pipeline - gcs_provide_file = gcs_hook.return_value.provide_file - self.dataflow.check_if_running = True - self.dataflow.multiple_jobs = True - - self.dataflow.execute(None) - - mock_callback_on_job_id.assert_called_once_with(on_new_job_id_callback=mock.ANY) - gcs_provide_file.assert_called_once_with(object_url=JAR_FILE) - expected_variables = { - "project": dataflow_hook_mock.return_value.project_id, - "stagingLocation": "gs://test/staging", - "jobName": JOB_NAME, - "region": TEST_LOCATION, - "output": "gs://test/output", - "labels": {"foo": "bar", "airflow-version": self.expected_airflow_version}, - } - assert expected_variables == is_job_dataflow_running_variables - job_name = dataflow_hook_mock.return_value.build_dataflow_job_name.return_value - expected_variables["jobName"] = job_name - start_java_mock.assert_called_once_with( - variables=expected_variables, - jar=gcs_provide_file.return_value.__enter__.return_value.name, - job_class=JOB_CLASS, - process_line_callback=mock_callback_on_job_id.return_value, - ) - dataflow_hook_mock.return_value.wait_for_done.assert_called_once_with( - job_id=mock.ANY, - job_name=job_name, - location=TEST_LOCATION, - multiple_jobs=True, - ) - - -class TestDataflowCreateJavaJobOperatorWithLocal: - def setup_method(self): - with warnings.catch_warnings(): - warnings.simplefilter("ignore", AirflowProviderDeprecationWarning) - self.dataflow = DataflowCreateJavaJobOperator( - task_id=TASK_ID, - jar=LOCAL_JAR_FILE, - job_name=JOB_NAME, - job_class=JOB_CLASS, - dataflow_default_options=DEFAULT_OPTIONS_JAVA, - options=ADDITIONAL_OPTIONS, - poll_sleep=POLL_SLEEP, - location=TEST_LOCATION, - ) - self.expected_airflow_version = "v" + airflow.version.version.replace(".", "-").replace("+", "-") - - def test_init(self): - """Test DataflowCreateJavaJobOperator instance is properly initialized.""" - assert self.dataflow.jar == LOCAL_JAR_FILE - - @mock.patch("airflow.providers.google.cloud.operators.dataflow.BeamHook") - @mock.patch("airflow.providers.google.cloud.operators.dataflow.DataflowHook") - def test_check_job_not_running_exec(self, dataflow_hook_mock, beam_hook_mock): - """Test DataflowHook is created and the right args are passed to - start_java_workflow with option to check if job is running - """ - is_job_dataflow_running_variables = None - - def set_is_job_dataflow_running_variables(*args, **kwargs): - nonlocal is_job_dataflow_running_variables - is_job_dataflow_running_variables = copy.deepcopy(kwargs.get("variables")) - - dataflow_running = dataflow_hook_mock.return_value.is_job_dataflow_running - dataflow_running.side_effect = set_is_job_dataflow_running_variables - dataflow_running.return_value = False - start_java_mock = beam_hook_mock.return_value.start_java_pipeline - self.dataflow.check_if_running = True - - self.dataflow.execute(None) - expected_variables = { - "project": dataflow_hook_mock.return_value.project_id, - "stagingLocation": "gs://test/staging", - "jobName": JOB_NAME, - "region": TEST_LOCATION, - "output": "gs://test/output", - "labels": {"foo": "bar", "airflow-version": self.expected_airflow_version}, - } - assert expected_variables == is_job_dataflow_running_variables - job_name = dataflow_hook_mock.return_value.build_dataflow_job_name.return_value - expected_variables["jobName"] = job_name - start_java_mock.assert_called_once_with( - variables=expected_variables, - jar=LOCAL_JAR_FILE, - job_class=JOB_CLASS, - process_line_callback=mock.ANY, - ) - dataflow_hook_mock.return_value.wait_for_done.assert_called_once_with( - job_id=mock.ANY, - job_name=job_name, - location=TEST_LOCATION, - multiple_jobs=False, - ) - - class TestDataflowTemplatedJobStartOperator: @pytest.fixture def sync_operator(self): diff --git a/providers/tests/google/cloud/operators/test_dataproc.py b/providers/tests/google/cloud/operators/test_dataproc.py index 2ec6ceb9babb..bbc7d05cd63d 100644 --- a/providers/tests/google/cloud/operators/test_dataproc.py +++ b/providers/tests/google/cloud/operators/test_dataproc.py @@ -19,7 +19,6 @@ import inspect from unittest import mock from unittest.mock import MagicMock, Mock, call -from uuid import UUID import pytest from google.api_core.exceptions import AlreadyExists, NotFound @@ -58,13 +57,7 @@ DataprocScaleClusterOperator, DataprocStartClusterOperator, DataprocStopClusterOperator, - DataprocSubmitHadoopJobOperator, - DataprocSubmitHiveJobOperator, DataprocSubmitJobOperator, - DataprocSubmitPigJobOperator, - DataprocSubmitPySparkJobOperator, - DataprocSubmitSparkJobOperator, - DataprocSubmitSparkSqlJobOperator, DataprocUpdateClusterOperator, InstanceFlexibilityPolicy, InstanceSelection, @@ -2229,429 +2222,6 @@ def test_instantiate_inline_workflow_operator_extra_links( assert ti.task.get_extra_links(ti, DataprocWorkflowLink.name) == DATAPROC_WORKFLOW_LINK_EXPECTED -class TestDataProcHiveOperator: - query = "define sin HiveUDF('sin');" - variables = {"key": "value"} - job_id = "uuid_id" - job_name = "simple" - job = { - "reference": {"project_id": GCP_PROJECT, "job_id": f"{job_name}_{job_id}"}, - "placement": {"cluster_name": "cluster-1"}, - "labels": {"airflow-version": AIRFLOW_VERSION_LABEL}, - "hive_job": {"query_list": {"queries": [query]}, "script_variables": variables}, - } - - @mock.patch(DATAPROC_PATH.format("DataprocHook")) - def test_deprecation_warning(self, mock_hook): - with pytest.warns(AirflowProviderDeprecationWarning) as warnings: - DataprocSubmitHiveJobOperator(task_id=TASK_ID, region=GCP_REGION, query="query") - assert_warning("DataprocSubmitJobOperator", warnings) - - @mock.patch(DATAPROC_PATH.format("uuid.uuid4")) - @mock.patch(DATAPROC_PATH.format("DataprocHook")) - def test_execute(self, mock_hook, mock_uuid): - mock_uuid.return_value = self.job_id - mock_hook.return_value.project_id = GCP_PROJECT - mock_hook.return_value.wait_for_job.return_value = None - mock_hook.return_value.submit_job.return_value.reference.job_id = self.job_id - - op = DataprocSubmitHiveJobOperator( - job_name=self.job_name, - task_id=TASK_ID, - region=GCP_REGION, - gcp_conn_id=GCP_CONN_ID, - query=self.query, - variables=self.variables, - impersonation_chain=IMPERSONATION_CHAIN, - ) - op.execute(context=MagicMock()) - mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) - mock_hook.return_value.submit_job.assert_called_once_with( - project_id=GCP_PROJECT, job=self.job, region=GCP_REGION - ) - mock_hook.return_value.wait_for_job.assert_called_once_with( - job_id=self.job_id, region=GCP_REGION, project_id=GCP_PROJECT - ) - - @mock.patch(DATAPROC_PATH.format("uuid.uuid4")) - @mock.patch(DATAPROC_PATH.format("DataprocHook")) - def test_builder(self, mock_hook, mock_uuid): - mock_hook.return_value.project_id = GCP_PROJECT - mock_uuid.return_value = self.job_id - - op = DataprocSubmitHiveJobOperator( - job_name=self.job_name, - task_id=TASK_ID, - region=GCP_REGION, - gcp_conn_id=GCP_CONN_ID, - query=self.query, - variables=self.variables, - ) - job = op.generate_job() - assert self.job == job - - -class TestDataProcPigOperator: - query = "define sin HiveUDF('sin');" - variables = {"key": "value"} - job_id = "uuid_id" - job_name = "simple" - job = { - "reference": {"project_id": GCP_PROJECT, "job_id": f"{job_name}_{job_id}"}, - "placement": {"cluster_name": "cluster-1"}, - "labels": {"airflow-version": AIRFLOW_VERSION_LABEL}, - "pig_job": {"query_list": {"queries": [query]}, "script_variables": variables}, - } - - @mock.patch(DATAPROC_PATH.format("DataprocHook")) - def test_deprecation_warning(self, mock_hook): - with pytest.warns(AirflowProviderDeprecationWarning) as warnings: - DataprocSubmitPigJobOperator(task_id=TASK_ID, region=GCP_REGION, query="query") - assert_warning("DataprocSubmitJobOperator", warnings) - - @mock.patch(DATAPROC_PATH.format("uuid.uuid4")) - @mock.patch(DATAPROC_PATH.format("DataprocHook")) - def test_execute(self, mock_hook, mock_uuid): - mock_uuid.return_value = self.job_id - mock_hook.return_value.project_id = GCP_PROJECT - mock_hook.return_value.wait_for_job.return_value = None - mock_hook.return_value.submit_job.return_value.reference.job_id = self.job_id - - op = DataprocSubmitPigJobOperator( - job_name=self.job_name, - task_id=TASK_ID, - region=GCP_REGION, - gcp_conn_id=GCP_CONN_ID, - query=self.query, - variables=self.variables, - impersonation_chain=IMPERSONATION_CHAIN, - ) - op.execute(context=MagicMock()) - mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) - mock_hook.return_value.submit_job.assert_called_once_with( - project_id=GCP_PROJECT, job=self.job, region=GCP_REGION - ) - mock_hook.return_value.wait_for_job.assert_called_once_with( - job_id=self.job_id, region=GCP_REGION, project_id=GCP_PROJECT - ) - - @mock.patch(DATAPROC_PATH.format("uuid.uuid4")) - @mock.patch(DATAPROC_PATH.format("DataprocHook")) - def test_builder(self, mock_hook, mock_uuid): - mock_hook.return_value.project_id = GCP_PROJECT - mock_uuid.return_value = self.job_id - - op = DataprocSubmitPigJobOperator( - job_name=self.job_name, - task_id=TASK_ID, - region=GCP_REGION, - gcp_conn_id=GCP_CONN_ID, - query=self.query, - variables=self.variables, - ) - job = op.generate_job() - assert self.job == job - - -class TestDataProcSparkSqlOperator: - query = "SHOW DATABASES;" - variables = {"key": "value"} - job_name = "simple" - job_id = "uuid_id" - job = { - "reference": {"project_id": GCP_PROJECT, "job_id": f"{job_name}_{job_id}"}, - "placement": {"cluster_name": "cluster-1"}, - "labels": {"airflow-version": AIRFLOW_VERSION_LABEL}, - "spark_sql_job": {"query_list": {"queries": [query]}, "script_variables": variables}, - } - other_project_job = { - "reference": {"project_id": "other-project", "job_id": f"{job_name}_{job_id}"}, - "placement": {"cluster_name": "cluster-1"}, - "labels": {"airflow-version": AIRFLOW_VERSION_LABEL}, - "spark_sql_job": {"query_list": {"queries": [query]}, "script_variables": variables}, - } - - @mock.patch(DATAPROC_PATH.format("DataprocHook")) - def test_deprecation_warning(self, mock_hook): - with pytest.warns(AirflowProviderDeprecationWarning) as warnings: - DataprocSubmitSparkSqlJobOperator(task_id=TASK_ID, region=GCP_REGION, query="query") - assert_warning("DataprocSubmitJobOperator", warnings) - - @mock.patch(DATAPROC_PATH.format("uuid.uuid4")) - @mock.patch(DATAPROC_PATH.format("DataprocHook")) - def test_execute(self, mock_hook, mock_uuid): - mock_uuid.return_value = self.job_id - mock_hook.return_value.project_id = GCP_PROJECT - mock_hook.return_value.wait_for_job.return_value = None - mock_hook.return_value.submit_job.return_value.reference.job_id = self.job_id - - op = DataprocSubmitSparkSqlJobOperator( - job_name=self.job_name, - task_id=TASK_ID, - region=GCP_REGION, - gcp_conn_id=GCP_CONN_ID, - query=self.query, - variables=self.variables, - impersonation_chain=IMPERSONATION_CHAIN, - ) - op.execute(context=MagicMock()) - mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) - mock_hook.return_value.submit_job.assert_called_once_with( - project_id=GCP_PROJECT, job=self.job, region=GCP_REGION - ) - mock_hook.return_value.wait_for_job.assert_called_once_with( - job_id=self.job_id, region=GCP_REGION, project_id=GCP_PROJECT - ) - - @mock.patch(DATAPROC_PATH.format("uuid.uuid4")) - @mock.patch(DATAPROC_PATH.format("DataprocHook")) - def test_execute_override_project_id(self, mock_hook, mock_uuid): - mock_uuid.return_value = self.job_id - mock_hook.return_value.project_id = GCP_PROJECT - mock_hook.return_value.wait_for_job.return_value = None - mock_hook.return_value.submit_job.return_value.reference.job_id = self.job_id - - op = DataprocSubmitSparkSqlJobOperator( - job_name=self.job_name, - project_id="other-project", - task_id=TASK_ID, - region=GCP_REGION, - gcp_conn_id=GCP_CONN_ID, - query=self.query, - variables=self.variables, - impersonation_chain=IMPERSONATION_CHAIN, - ) - op.execute(context=MagicMock()) - mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) - mock_hook.return_value.submit_job.assert_called_once_with( - project_id="other-project", job=self.other_project_job, region=GCP_REGION - ) - mock_hook.return_value.wait_for_job.assert_called_once_with( - job_id=self.job_id, region=GCP_REGION, project_id="other-project" - ) - - @mock.patch(DATAPROC_PATH.format("uuid.uuid4")) - @mock.patch(DATAPROC_PATH.format("DataprocHook")) - def test_builder(self, mock_hook, mock_uuid): - mock_hook.return_value.project_id = GCP_PROJECT - mock_uuid.return_value = self.job_id - - op = DataprocSubmitSparkSqlJobOperator( - job_name=self.job_name, - task_id=TASK_ID, - region=GCP_REGION, - gcp_conn_id=GCP_CONN_ID, - query=self.query, - variables=self.variables, - ) - job = op.generate_job() - assert self.job == job - - -class TestDataProcSparkOperator(DataprocJobTestBase): - @classmethod - def setup_class(cls): - if AIRFLOW_V_3_0_PLUS: - cls.extra_links_expected_calls = [ - call.ti.xcom_push(key="conf", value=DATAPROC_JOB_CONF_EXPECTED), - call.hook().wait_for_job(job_id=TEST_JOB_ID, region=GCP_REGION, project_id=GCP_PROJECT), - ] - else: - cls.extra_links_expected_calls = [ - call.ti.xcom_push(key="conf", value=DATAPROC_JOB_CONF_EXPECTED, execution_date=None), - call.hook().wait_for_job(job_id=TEST_JOB_ID, region=GCP_REGION, project_id=GCP_PROJECT), - ] - - main_class = "org.apache.spark.examples.SparkPi" - jars = ["file:///usr/lib/spark/examples/jars/spark-examples.jar"] - job_name = "simple" - job = { - "reference": { - "project_id": GCP_PROJECT, - "job_id": f"{job_name}_{TEST_JOB_ID}", - }, - "placement": {"cluster_name": "cluster-1"}, - "labels": {"airflow-version": AIRFLOW_VERSION_LABEL}, - "spark_job": {"jar_file_uris": jars, "main_class": main_class}, - } - - @mock.patch(DATAPROC_PATH.format("DataprocHook")) - def test_deprecation_warning(self, mock_hook): - with pytest.warns(AirflowProviderDeprecationWarning) as warnings: - DataprocSubmitSparkJobOperator( - task_id=TASK_ID, region=GCP_REGION, main_class=self.main_class, dataproc_jars=self.jars - ) - assert_warning("DataprocSubmitJobOperator", warnings) - - @mock.patch(DATAPROC_PATH.format("uuid.uuid4")) - @mock.patch(DATAPROC_PATH.format("DataprocHook")) - def test_execute(self, mock_hook, mock_uuid): - mock_uuid.return_value = TEST_JOB_ID - mock_hook.return_value.project_id = GCP_PROJECT - mock_uuid.return_value = TEST_JOB_ID - mock_hook.return_value.submit_job.return_value.reference.job_id = TEST_JOB_ID - self.extra_links_manager_mock.attach_mock(mock_hook, "hook") - - op = DataprocSubmitSparkJobOperator( - job_name=self.job_name, - task_id=TASK_ID, - region=GCP_REGION, - gcp_conn_id=GCP_CONN_ID, - main_class=self.main_class, - dataproc_jars=self.jars, - ) - job = op.generate_job() - assert self.job == job - - op.execute(context=self.mock_context) - if AIRFLOW_V_3_0_PLUS: - self.mock_ti.xcom_push.assert_called_once_with(key="conf", value=DATAPROC_JOB_CONF_EXPECTED) - else: - self.mock_ti.xcom_push.assert_called_once_with( - key="conf", value=DATAPROC_JOB_CONF_EXPECTED, execution_date=None - ) - - # Test whether xcom push occurs before polling for job - self.extra_links_manager_mock.assert_has_calls(self.extra_links_expected_calls, any_order=False) - - -@pytest.mark.db_test -@pytest.mark.need_serialized_dag -@mock.patch(DATAPROC_PATH.format("DataprocHook")) -def test_submit_spark_job_operator_extra_links(mock_hook, dag_maker, create_task_instance_of_operator): - mock_hook.return_value.project_id = GCP_PROJECT - - ti = create_task_instance_of_operator( - DataprocSubmitSparkJobOperator, - dag_id=TEST_DAG_ID, - execution_date=DEFAULT_DATE, - task_id=TASK_ID, - region=GCP_REGION, - gcp_conn_id=GCP_CONN_ID, - main_class="org.apache.spark.examples.SparkPi", - dataproc_jars=["file:///usr/lib/spark/examples/jars/spark-examples.jar"], - ) - - serialized_dag = dag_maker.get_serialized_data() - deserialized_dag = SerializedDAG.from_dict(serialized_dag) - deserialized_task = deserialized_dag.task_dict[TASK_ID] - - # Assert operator links for serialized DAG - deserialized_dag = SerializedDAG.deserialize_dag(serialized_dag["dag"]) - operator_extra_link = deserialized_dag.tasks[0].operator_extra_links[0] - assert operator_extra_link.name == "Dataproc resource" - - # Assert operator link types are preserved during deserialization - assert isinstance(deserialized_task.operator_extra_links[0], DataprocLink) - - # Assert operator link is empty when no XCom push occurred - assert ti.task.get_extra_links(ti, DataprocLink.name) == "" - - # Assert operator link is empty for deserialized task when no XCom push occurred - assert deserialized_task.get_extra_links(ti, DataprocLink.name) == "" - - ti.xcom_push(key="conf", value=DATAPROC_JOB_CONF_EXPECTED) - - # Assert operator links after task execution - assert ti.task.get_extra_links(ti, DataprocLink.name) == DATAPROC_JOB_LINK_EXPECTED - - # Assert operator links are preserved in deserialized tasks - link = deserialized_task.get_extra_links(ti, DataprocLink.name) - assert link == DATAPROC_JOB_LINK_EXPECTED - - -class TestDataProcHadoopOperator: - args = ["wordcount", "gs://pub/shakespeare/rose.txt"] - jar = "file:///usr/lib/spark/examples/jars/spark-examples.jar" - job_name = "simple" - job_id = "uuid_id" - job = { - "reference": {"project_id": GCP_PROJECT, "job_id": f"{job_name}_{job_id}"}, - "placement": {"cluster_name": "cluster-1"}, - "labels": {"airflow-version": AIRFLOW_VERSION_LABEL}, - "hadoop_job": {"main_jar_file_uri": jar, "args": args}, - } - - @mock.patch(DATAPROC_PATH.format("DataprocHook")) - def test_deprecation_warning(self, mock_hook): - with pytest.warns(AirflowProviderDeprecationWarning) as warnings: - DataprocSubmitHadoopJobOperator( - task_id=TASK_ID, region=GCP_REGION, main_jar=self.jar, arguments=self.args - ) - assert_warning("DataprocSubmitJobOperator", warnings) - - @mock.patch(DATAPROC_PATH.format("uuid.uuid4")) - @mock.patch(DATAPROC_PATH.format("DataprocHook")) - def test_execute(self, mock_hook, mock_uuid): - mock_uuid.return_value = self.job_id - mock_hook.return_value.project_id = GCP_PROJECT - mock_uuid.return_value = self.job_id - - op = DataprocSubmitHadoopJobOperator( - job_name=self.job_name, - task_id=TASK_ID, - region=GCP_REGION, - gcp_conn_id=GCP_CONN_ID, - main_jar=self.jar, - arguments=self.args, - ) - job = op.generate_job() - assert self.job == job - - -class TestDataProcPySparkOperator: - uri = "gs://{}/{}" - job_id = "uuid_id" - job_name = "simple" - job = { - "reference": {"project_id": GCP_PROJECT, "job_id": f"{job_name}_{job_id}"}, - "placement": {"cluster_name": "cluster-1"}, - "labels": {"airflow-version": AIRFLOW_VERSION_LABEL}, - "pyspark_job": {"main_python_file_uri": uri}, - } - - @mock.patch(DATAPROC_PATH.format("DataprocHook")) - def test_deprecation_warning(self, mock_hook): - with pytest.warns(AirflowProviderDeprecationWarning) as warnings: - DataprocSubmitPySparkJobOperator(task_id=TASK_ID, region=GCP_REGION, main=self.uri) - assert_warning("DataprocSubmitJobOperator", warnings) - - @mock.patch(DATAPROC_PATH.format("uuid.uuid4")) - @mock.patch(DATAPROC_PATH.format("DataprocHook")) - def test_execute(self, mock_hook, mock_uuid): - mock_hook.return_value.project_id = GCP_PROJECT - mock_uuid.return_value = self.job_id - - op = DataprocSubmitPySparkJobOperator( - job_name=self.job_name, - task_id=TASK_ID, - region=GCP_REGION, - gcp_conn_id=GCP_CONN_ID, - main=self.uri, - ) - job = op.generate_job() - assert self.job == job - - @pytest.mark.parametrize( - "filename", - [ - pytest.param("/foo/bar/baz.py", id="absolute"), - pytest.param("foo/bar/baz.py", id="relative"), - pytest.param("baz.py", id="base-filename"), - pytest.param(r"C:\foo\bar\baz.py", id="windows-path"), - ], - ) - def test_generate_temp_filename(self, filename, time_machine): - time_machine.move_to(datetime(2024, 2, 29, 1, 2, 3), tick=False) - with mock.patch( - DATAPROC_PATH.format("uuid.uuid4"), return_value=UUID("12345678-0000-4000-0000-000000000000") - ): - assert ( - DataprocSubmitPySparkJobOperator._generate_temp_filename(filename) - == "20240229010203_12345678_baz.py" - ) - - class TestDataprocCreateWorkflowTemplateOperator: @mock.patch(DATAPROC_PATH.format("DataprocHook")) def test_execute(self, mock_hook): diff --git a/providers/tests/google/cloud/operators/test_vertex_ai.py b/providers/tests/google/cloud/operators/test_vertex_ai.py index f79542006e26..55f09da8106a 100644 --- a/providers/tests/google/cloud/operators/test_vertex_ai.py +++ b/providers/tests/google/cloud/operators/test_vertex_ai.py @@ -32,7 +32,6 @@ CreateAutoMLForecastingTrainingJobOperator, CreateAutoMLImageTrainingJobOperator, CreateAutoMLTabularTrainingJobOperator, - CreateAutoMLTextTrainingJobOperator, CreateAutoMLVideoTrainingJobOperator, DeleteAutoMLTrainingJobOperator, ListAutoMLTrainingJobOperator, @@ -246,10 +245,7 @@ def test_execute(self, mock_hook, mock_dataset): dataset_id=TEST_DATASET_ID, parent_model=TEST_PARENT_MODEL, ) - with pytest.warns( - AirflowProviderDeprecationWarning, match=SYNC_DEPRECATION_WARNING.format("01.10.2024") - ): - op.execute(context={"ti": mock.MagicMock()}) + op.execute(context={"ti": mock.MagicMock()}) mock_dataset.assert_called_once_with(name=TEST_DATASET_ID) mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) mock_hook.return_value.create_custom_container_training_job.assert_called_once_with( @@ -337,10 +333,7 @@ def test_execute__parent_model_version_index_is_removed(self, mock_hook, mock_da dataset_id=TEST_DATASET_ID, parent_model=VERSIONED_TEST_PARENT_MODEL, ) - with pytest.warns( - AirflowProviderDeprecationWarning, match=SYNC_DEPRECATION_WARNING.format("01.10.2024") - ): - op.execute(context={"ti": mock.MagicMock()}) + op.execute(context={"ti": mock.MagicMock()}) mock_hook.return_value.create_custom_container_training_job.assert_called_once_with( staging_bucket=STAGING_BUCKET, display_name=DISPLAY_NAME, @@ -421,10 +414,7 @@ def test_execute_enters_deferred_state(self, mock_hook): ) mock_hook.return_value.exists.return_value = False with pytest.raises(TaskDeferred) as exc: - with pytest.warns( - AirflowProviderDeprecationWarning, match=SYNC_DEPRECATION_WARNING.format("01.10.2024") - ): - task.execute(context={"ti": mock.MagicMock()}) + task.execute(context={"ti": mock.MagicMock()}) assert isinstance( exc.value.trigger, CustomContainerTrainingJobTrigger ), "Trigger is not a CustomContainerTrainingJobTrigger" @@ -577,10 +567,7 @@ def test_execute(self, mock_hook, mock_dataset): dataset_id=TEST_DATASET_ID, parent_model=TEST_PARENT_MODEL, ) - with pytest.warns( - AirflowProviderDeprecationWarning, match=SYNC_DEPRECATION_WARNING.format("01.10.2024") - ): - op.execute(context={"ti": mock.MagicMock()}) + op.execute(context={"ti": mock.MagicMock()}) mock_dataset.assert_called_once_with(name=TEST_DATASET_ID) mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) mock_hook.return_value.create_custom_python_package_training_job.assert_called_once_with( @@ -670,10 +657,7 @@ def test_execute__parent_model_version_index_is_removed(self, mock_hook, mock_da dataset_id=TEST_DATASET_ID, parent_model=VERSIONED_TEST_PARENT_MODEL, ) - with pytest.warns( - AirflowProviderDeprecationWarning, match=SYNC_DEPRECATION_WARNING.format("01.10.2024") - ): - op.execute(context={"ti": mock.MagicMock()}) + op.execute(context={"ti": mock.MagicMock()}) mock_hook.return_value.create_custom_python_package_training_job.assert_called_once_with( staging_bucket=STAGING_BUCKET, display_name=DISPLAY_NAME, @@ -756,10 +740,7 @@ def test_execute_enters_deferred_state(self, mock_hook): ) mock_hook.return_value.exists.return_value = False with pytest.raises(TaskDeferred) as exc: - with pytest.warns( - AirflowProviderDeprecationWarning, match=SYNC_DEPRECATION_WARNING.format("01.10.2024") - ): - task.execute(context={"ti": mock.MagicMock()}) + task.execute(context={"ti": mock.MagicMock()}) assert isinstance( exc.value.trigger, CustomPythonPackageTrainingJobTrigger ), "Trigger is not a CustomPythonPackageTrainingJobTrigger" @@ -911,10 +892,7 @@ def test_execute(self, mock_hook, mock_dataset): dataset_id=TEST_DATASET_ID, parent_model=TEST_PARENT_MODEL, ) - with pytest.warns( - AirflowProviderDeprecationWarning, match=SYNC_DEPRECATION_WARNING.format("01.10.2024") - ): - op.execute(context={"ti": mock.MagicMock()}) + op.execute(context={"ti": mock.MagicMock()}) mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) mock_dataset.assert_called_once_with(name=TEST_DATASET_ID) mock_hook.return_value.create_custom_training_job.assert_called_once_with( @@ -997,10 +975,7 @@ def test_execute__parent_model_version_index_is_removed(self, mock_hook, mock_da dataset_id=TEST_DATASET_ID, parent_model=VERSIONED_TEST_PARENT_MODEL, ) - with pytest.warns( - AirflowProviderDeprecationWarning, match=SYNC_DEPRECATION_WARNING.format("01.10.2024") - ): - op.execute(context={"ti": mock.MagicMock()}) + op.execute(context={"ti": mock.MagicMock()}) mock_hook.return_value.create_custom_training_job.assert_called_once_with( staging_bucket=STAGING_BUCKET, display_name=DISPLAY_NAME, @@ -1076,10 +1051,7 @@ def test_execute_enters_deferred_state(self, mock_hook): ) mock_hook.return_value.exists.return_value = False with pytest.raises(TaskDeferred) as exc: - with pytest.warns( - AirflowProviderDeprecationWarning, match=SYNC_DEPRECATION_WARNING.format("01.10.2024") - ): - task.execute(context={"ti": mock.MagicMock()}) + task.execute(context={"ti": mock.MagicMock()}) assert isinstance( exc.value.trigger, CustomTrainingJobTrigger ), "Trigger is not a CustomTrainingJobTrigger" @@ -1828,102 +1800,6 @@ def test_execute__parent_model_version_index_is_removed(self, mock_hook, mock_da ) -class TestVertexAICreateAutoMLTextTrainingJobOperator: - @mock.patch("google.cloud.aiplatform.datasets.TextDataset") - @mock.patch(VERTEX_AI_PATH.format("auto_ml.AutoMLHook")) - def test_execute(self, mock_hook, mock_dataset): - mock_hook.return_value.create_auto_ml_text_training_job.return_value = (None, "training_id") - with pytest.warns(AirflowProviderDeprecationWarning): - op = CreateAutoMLTextTrainingJobOperator( - task_id=TASK_ID, - gcp_conn_id=GCP_CONN_ID, - impersonation_chain=IMPERSONATION_CHAIN, - display_name=DISPLAY_NAME, - dataset_id=TEST_DATASET_ID, - prediction_type=None, - multi_label=False, - sentiment_max=10, - sync=True, - region=GCP_LOCATION, - project_id=GCP_PROJECT, - parent_model=TEST_PARENT_MODEL, - ) - op.execute(context={"ti": mock.MagicMock()}) - mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) - mock_dataset.assert_called_once_with(dataset_name=TEST_DATASET_ID) - mock_hook.return_value.create_auto_ml_text_training_job.assert_called_once_with( - project_id=GCP_PROJECT, - region=GCP_LOCATION, - display_name=DISPLAY_NAME, - dataset=mock_dataset.return_value, - parent_model=TEST_PARENT_MODEL, - prediction_type=None, - multi_label=False, - sentiment_max=10, - labels=None, - training_encryption_spec_key_name=None, - model_encryption_spec_key_name=None, - training_fraction_split=None, - validation_fraction_split=None, - test_fraction_split=None, - training_filter_split=None, - validation_filter_split=None, - test_filter_split=None, - model_display_name=None, - model_labels=None, - sync=True, - is_default_version=None, - model_version_aliases=None, - model_version_description=None, - ) - - @mock.patch("google.cloud.aiplatform.datasets.TextDataset") - @mock.patch(VERTEX_AI_PATH.format("auto_ml.AutoMLHook")) - def test_execute__parent_model_version_index_is_removed(self, mock_hook, mock_dataset): - mock_hook.return_value.create_auto_ml_text_training_job.return_value = (None, "training_id") - with pytest.warns(AirflowProviderDeprecationWarning): - op = CreateAutoMLTextTrainingJobOperator( - task_id=TASK_ID, - gcp_conn_id=GCP_CONN_ID, - impersonation_chain=IMPERSONATION_CHAIN, - display_name=DISPLAY_NAME, - dataset_id=TEST_DATASET_ID, - prediction_type=None, - multi_label=False, - sentiment_max=10, - sync=True, - region=GCP_LOCATION, - project_id=GCP_PROJECT, - parent_model=VERSIONED_TEST_PARENT_MODEL, - ) - op.execute(context={"ti": mock.MagicMock()}) - mock_hook.return_value.create_auto_ml_text_training_job.assert_called_once_with( - project_id=GCP_PROJECT, - region=GCP_LOCATION, - display_name=DISPLAY_NAME, - dataset=mock_dataset.return_value, - parent_model=TEST_PARENT_MODEL, - prediction_type=None, - multi_label=False, - sentiment_max=10, - labels=None, - training_encryption_spec_key_name=None, - model_encryption_spec_key_name=None, - training_fraction_split=None, - validation_fraction_split=None, - test_fraction_split=None, - training_filter_split=None, - validation_filter_split=None, - test_filter_split=None, - model_display_name=None, - model_labels=None, - sync=True, - is_default_version=None, - model_version_aliases=None, - model_version_description=None, - ) - - class TestVertexAICreateAutoMLVideoTrainingJobOperator: @mock.patch("google.cloud.aiplatform.datasets.VideoDataset") @mock.patch(VERTEX_AI_PATH.format("auto_ml.AutoMLHook")) @@ -2120,11 +1996,7 @@ def test_execute(self, mock_hook, mock_link_persist): batch_size=TEST_BATCH_SIZE, ) context = {"ti": mock.MagicMock()} - with pytest.warns( - AirflowProviderDeprecationWarning, - match=SYNC_DEPRECATION_WARNING.format("28.08.2024"), - ): - op.execute(context=context) + op.execute(context=context) mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) mock_hook.return_value.submit_batch_prediction_job.assert_called_once_with( @@ -2504,17 +2376,12 @@ def test_execute(self, mock_hook, to_dict_mock): staging_bucket=STAGING_BUCKET, display_name=DISPLAY_NAME, worker_pool_specs=[], - sync=False, parameter_spec={}, metric_spec={}, max_trial_count=15, parallel_trial_count=3, ) - with pytest.warns( - AirflowProviderDeprecationWarning, - match=SYNC_DEPRECATION_WARNING.format("01.09.2024"), - ): - op.execute(context={"ti": mock.MagicMock()}) + op.execute(context={"ti": mock.MagicMock()}) mock_hook.assert_called_once_with(gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN) mock_hook.return_value.create_hyperparameter_tuning_job.assert_called_once_with( project_id=GCP_PROJECT, @@ -2558,47 +2425,15 @@ def test_deferrable(self, mock_hook, mock_defer): staging_bucket=STAGING_BUCKET, display_name=DISPLAY_NAME, worker_pool_specs=[], - sync=False, parameter_spec={}, metric_spec={}, max_trial_count=15, parallel_trial_count=3, deferrable=True, ) - with pytest.warns( - AirflowProviderDeprecationWarning, - match=SYNC_DEPRECATION_WARNING.format("01.09.2024"), - ): - op.execute(context={"ti": mock.MagicMock()}) + op.execute(context={"ti": mock.MagicMock()}) mock_defer.assert_called_once() - @pytest.mark.db_test - def test_deferrable_sync_error(self): - op = CreateHyperparameterTuningJobOperator( - task_id=TASK_ID, - gcp_conn_id=GCP_CONN_ID, - impersonation_chain=IMPERSONATION_CHAIN, - region=GCP_LOCATION, - project_id=GCP_PROJECT, - staging_bucket=STAGING_BUCKET, - display_name=DISPLAY_NAME, - worker_pool_specs=[], - sync=True, - parameter_spec={}, - metric_spec={}, - max_trial_count=15, - parallel_trial_count=3, - deferrable=True, - ) - with ( - pytest.raises(AirflowException), - pytest.warns( - AirflowProviderDeprecationWarning, - match=SYNC_DEPRECATION_WARNING.format("01.09.2024"), - ), - ): - op.execute(context={"ti": mock.MagicMock()}) - @mock.patch(VERTEX_AI_PATH.format("hyperparameter_tuning_job.HyperparameterTuningJobHook")) def test_execute_complete(self, mock_hook): test_job_id = "test_job_id" @@ -2620,7 +2455,6 @@ def test_execute_complete(self, mock_hook): staging_bucket=STAGING_BUCKET, display_name=DISPLAY_NAME, worker_pool_specs=[], - sync=False, parameter_spec={}, metric_spec={}, max_trial_count=15, @@ -2646,7 +2480,6 @@ def test_execute_complete_error(self): staging_bucket=STAGING_BUCKET, display_name=DISPLAY_NAME, worker_pool_specs=[], - sync=False, parameter_spec={}, metric_spec={}, max_trial_count=15, diff --git a/providers/tests/google/cloud/sensors/test_bigquery.py b/providers/tests/google/cloud/sensors/test_bigquery.py index b30edee76520..91a98970f93c 100644 --- a/providers/tests/google/cloud/sensors/test_bigquery.py +++ b/providers/tests/google/cloud/sensors/test_bigquery.py @@ -22,12 +22,9 @@ from airflow.exceptions import ( AirflowException, - AirflowProviderDeprecationWarning, TaskDeferred, ) from airflow.providers.google.cloud.sensors.bigquery import ( - BigQueryTableExistenceAsyncSensor, - BigQueryTableExistencePartitionAsyncSensor, BigQueryTableExistenceSensor, BigQueryTablePartitionExistenceSensor, ) @@ -88,7 +85,7 @@ def test_table_existence_sensor_finish_before_deferred(self, mock_defer, mock_ho def test_execute_deferred(self, mock_hook): """ Asserts that a task is deferred and a BigQueryTableExistenceTrigger will be fired - when the BigQueryTableExistenceAsyncSensor is executed. + when the BigQueryTableExistenceSensor is executed. """ task = BigQueryTableExistenceSensor( task_id="check_table_exists", @@ -258,131 +255,3 @@ def context(): """ context = {} return context - - -class TestBigQueryTableExistenceAsyncSensor: - @mock.patch("airflow.providers.google.cloud.sensors.bigquery.BigQueryHook") - def test_big_query_table_existence_sensor_async(self, mock_hook): - """ - Asserts that a task is deferred and a BigQueryTableExistenceTrigger will be fired - when the BigQueryTableExistenceAsyncSensor is executed. - """ - with pytest.warns(AirflowProviderDeprecationWarning): - task = BigQueryTableExistenceAsyncSensor( - task_id="check_table_exists", - project_id=TEST_PROJECT_ID, - dataset_id=TEST_DATASET_ID, - table_id=TEST_TABLE_ID, - ) - mock_hook.return_value.table_exists.return_value = False - with pytest.raises(TaskDeferred) as exc: - task.execute(context={}) - assert isinstance( - exc.value.trigger, BigQueryTableExistenceTrigger - ), "Trigger is not a BigQueryTableExistenceTrigger" - - def test_big_query_table_existence_sensor_async_execute_failure(self): - """Tests that an expected_exception is raised in case of error event""" - with pytest.warns(AirflowProviderDeprecationWarning): - task = BigQueryTableExistenceAsyncSensor( - task_id="task-id", - project_id=TEST_PROJECT_ID, - dataset_id=TEST_DATASET_ID, - table_id=TEST_TABLE_ID, - ) - with pytest.raises(AirflowException): - task.execute_complete(context={}, event={"status": "error", "message": "test failure message"}) - - def test_big_query_table_existence_sensor_async_execute_complete(self): - """Asserts that logging occurs as expected""" - with pytest.warns(AirflowProviderDeprecationWarning): - task = BigQueryTableExistenceAsyncSensor( - task_id="task-id", - project_id=TEST_PROJECT_ID, - dataset_id=TEST_DATASET_ID, - table_id=TEST_TABLE_ID, - ) - table_uri = f"{TEST_PROJECT_ID}:{TEST_DATASET_ID}.{TEST_TABLE_ID}" - with mock.patch.object(task.log, "info") as mock_log_info: - task.execute_complete(context={}, event={"status": "success", "message": "Job completed"}) - mock_log_info.assert_called_with("Sensor checks existence of table: %s", table_uri) - - def test_big_query_sensor_async_execute_complete_event_none( - self, - ): - """Asserts that logging occurs as expected""" - with pytest.warns(AirflowProviderDeprecationWarning): - task = BigQueryTableExistenceAsyncSensor( - task_id="task-id", - project_id=TEST_PROJECT_ID, - dataset_id=TEST_DATASET_ID, - table_id=TEST_TABLE_ID, - ) - with pytest.raises(AirflowException): - task.execute_complete(context={}, event=None) - - -class TestBigQueryTableExistencePartitionAsyncSensor: - @mock.patch("airflow.providers.google.cloud.sensors.bigquery.BigQueryHook") - def test_big_query_table_existence_partition_sensor_async(self, mock_hook): - """ - Asserts that a task is deferred and a BigQueryTablePartitionExistenceTrigger will be fired - when the BigQueryTableExistencePartitionAsyncSensor is executed. - """ - with pytest.warns(AirflowProviderDeprecationWarning): - task = BigQueryTableExistencePartitionAsyncSensor( - task_id="test_task_id", - project_id=TEST_PROJECT_ID, - dataset_id=TEST_DATASET_ID, - table_id=TEST_TABLE_ID, - partition_id=TEST_PARTITION_ID, - ) - mock_hook.return_value.table_partition_exists.return_value = False - with pytest.raises(TaskDeferred) as exc: - task.execute(mock.MagicMock()) - assert isinstance( - exc.value.trigger, BigQueryTablePartitionExistenceTrigger - ), "Trigger is not a BigQueryTablePartitionExistenceTrigger" - - def test_big_query_table_existence_partition_sensor_async_execute_failure(self): - """Tests that an expected exception is raised in case of error event""" - with pytest.warns(AirflowProviderDeprecationWarning): - task = BigQueryTableExistencePartitionAsyncSensor( - task_id="test_task_id", - project_id=TEST_PROJECT_ID, - dataset_id=TEST_DATASET_ID, - table_id=TEST_TABLE_ID, - partition_id=TEST_PARTITION_ID, - ) - with pytest.raises(AirflowException): - task.execute_complete(context={}, event={"status": "error", "message": "test failure message"}) - - def test_big_query_table_existence_partition_sensor_async_execute_complete_event_none(self): - """Asserts that logging occurs as expected""" - with pytest.warns(AirflowProviderDeprecationWarning): - task = BigQueryTableExistencePartitionAsyncSensor( - task_id="task-id", - project_id=TEST_PROJECT_ID, - dataset_id=TEST_DATASET_ID, - table_id=TEST_TABLE_ID, - partition_id=TEST_PARTITION_ID, - ) - with pytest.raises(AirflowException, match="No event received in trigger callback"): - task.execute_complete(context={}, event=None) - - def test_big_query_table_existence_partition_sensor_async_execute_complete(self): - """Asserts that logging occurs as expected""" - with pytest.warns(AirflowProviderDeprecationWarning): - task = BigQueryTableExistencePartitionAsyncSensor( - task_id="task-id", - project_id=TEST_PROJECT_ID, - dataset_id=TEST_DATASET_ID, - table_id=TEST_TABLE_ID, - partition_id=TEST_PARTITION_ID, - ) - table_uri = f"{TEST_PROJECT_ID}:{TEST_DATASET_ID}.{TEST_TABLE_ID}" - with mock.patch.object(task.log, "info") as mock_log_info: - task.execute_complete(context={}, event={"status": "success", "message": "test"}) - mock_log_info.assert_called_with( - 'Sensor checks existence of partition: "%s" in table: %s', TEST_PARTITION_ID, table_uri - ) diff --git a/providers/tests/google/cloud/sensors/test_cloud_composer.py b/providers/tests/google/cloud/sensors/test_cloud_composer.py index 1e21898ae6c2..c69df530a2f6 100644 --- a/providers/tests/google/cloud/sensors/test_cloud_composer.py +++ b/providers/tests/google/cloud/sensors/test_cloud_composer.py @@ -21,20 +21,7 @@ from datetime import datetime from unittest import mock -import pytest - -from airflow.exceptions import ( - AirflowException, - AirflowProviderDeprecationWarning, - TaskDeferred, -) -from airflow.providers.google.cloud.sensors.cloud_composer import ( - CloudComposerDAGRunSensor, - CloudComposerEnvironmentSensor, -) -from airflow.providers.google.cloud.triggers.cloud_composer import ( - CloudComposerExecutionTrigger, -) +from airflow.providers.google.cloud.sensors.cloud_composer import CloudComposerDAGRunSensor TEST_PROJECT_ID = "test_project_id" TEST_OPERATION_NAME = "test_operation_name" @@ -59,55 +46,6 @@ } -class TestCloudComposerEnvironmentSensor: - @pytest.mark.db_test - def test_cloud_composer_existence_sensor_async(self): - """ - Asserts that a task is deferred and a CloudComposerExecutionTrigger will be fired - when the CloudComposerEnvironmentSensor is executed. - """ - with pytest.warns(AirflowProviderDeprecationWarning): - task = CloudComposerEnvironmentSensor( - task_id="task_id", - project_id=TEST_PROJECT_ID, - region=TEST_REGION, - operation_name=TEST_OPERATION_NAME, - ) - with pytest.raises(TaskDeferred) as exc: - task.execute(context={}) - assert isinstance( - exc.value.trigger, CloudComposerExecutionTrigger - ), "Trigger is not a CloudComposerExecutionTrigger" - - def test_cloud_composer_existence_sensor_async_execute_failure( - self, - ): - """Tests that an expected exception is raised in case of error event.""" - with pytest.warns(AirflowProviderDeprecationWarning): - task = CloudComposerEnvironmentSensor( - task_id="task_id", - project_id=TEST_PROJECT_ID, - region=TEST_REGION, - operation_name=TEST_OPERATION_NAME, - ) - with pytest.raises(AirflowException, match="No event received in trigger callback"): - task.execute_complete(context={}, event=None) - - def test_cloud_composer_existence_sensor_async_execute_complete(self): - """Asserts that logging occurs as expected""" - with pytest.warns(AirflowProviderDeprecationWarning): - task = CloudComposerEnvironmentSensor( - task_id="task_id", - project_id=TEST_PROJECT_ID, - region=TEST_REGION, - operation_name=TEST_OPERATION_NAME, - ) - with mock.patch.object(task.log, "info"): - task.execute_complete( - context={}, event={"operation_done": True, "operation_name": TEST_OPERATION_NAME} - ) - - class TestCloudComposerDAGRunSensor: @mock.patch("airflow.providers.google.cloud.sensors.cloud_composer.ExecuteAirflowCommandResponse.to_dict") @mock.patch("airflow.providers.google.cloud.sensors.cloud_composer.CloudComposerHook") diff --git a/providers/tests/google/cloud/sensors/test_gcs.py b/providers/tests/google/cloud/sensors/test_gcs.py index 8e9a5d3bba85..bcec4f6220a8 100644 --- a/providers/tests/google/cloud/sensors/test_gcs.py +++ b/providers/tests/google/cloud/sensors/test_gcs.py @@ -23,13 +23,9 @@ import pytest from google.cloud.storage.retry import DEFAULT_RETRY -from airflow.exceptions import ( - AirflowProviderDeprecationWarning, - TaskDeferred, -) +from airflow.exceptions import TaskDeferred from airflow.models.dag import DAG, AirflowException from airflow.providers.google.cloud.sensors.gcs import ( - GCSObjectExistenceAsyncSensor, GCSObjectExistenceSensor, GCSObjectsWithPrefixExistenceSensor, GCSObjectUpdateSensor, @@ -205,51 +201,6 @@ def test_gcs_object_existence_sensor_execute_complete_return_value(self): assert return_value, True -class TestGoogleCloudStorageObjectAsyncSensor: - @mock.patch("airflow.providers.google.cloud.sensors.gcs.GCSHook") - def test_gcs_object_existence_async_sensor(self, mock_hook): - """ - Asserts that a task is deferred and a GCSBlobTrigger will be fired - when the GCSObjectExistenceAsyncSensor is executed. - """ - with pytest.warns(AirflowProviderDeprecationWarning): - task = GCSObjectExistenceAsyncSensor( - task_id="task-id", - bucket=TEST_BUCKET, - object=TEST_OBJECT, - google_cloud_conn_id=TEST_GCP_CONN_ID, - ) - mock_hook.return_value.exists.return_value = False - with pytest.raises(TaskDeferred) as exc: - task.execute({}) - assert isinstance(exc.value.trigger, GCSBlobTrigger), "Trigger is not a GCSBlobTrigger" - - def test_gcs_object_existence_async_sensor_execute_failure(self): - """Tests that an AirflowException is raised in case of error event""" - with pytest.warns(AirflowProviderDeprecationWarning): - task = GCSObjectExistenceAsyncSensor( - task_id="task-id", - bucket=TEST_BUCKET, - object=TEST_OBJECT, - google_cloud_conn_id=TEST_GCP_CONN_ID, - ) - with pytest.raises(AirflowException): - task.execute_complete(context=None, event={"status": "error", "message": "test failure message"}) - - def test_gcs_object_existence_async_sensor_execute_complete(self): - """Asserts that logging occurs as expected""" - with pytest.warns(AirflowProviderDeprecationWarning): - task = GCSObjectExistenceAsyncSensor( - task_id="task-id", - bucket=TEST_BUCKET, - object=TEST_OBJECT, - google_cloud_conn_id=TEST_GCP_CONN_ID, - ) - with mock.patch.object(task.log, "info") as mock_log_info: - task.execute_complete(context=None, event={"status": "success", "message": "Job completed"}) - mock_log_info.assert_called_with("File %s was found in bucket %s.", TEST_OBJECT, TEST_BUCKET) - - class TestGoogleCloudStorageObjectUpdatedSensor: @mock.patch("airflow.providers.google.cloud.sensors.gcs.GCSHook") def test_should_pass_argument_to_hook(self, mock_hook): diff --git a/providers/tests/google/cloud/triggers/test_kubernetes_engine.py b/providers/tests/google/cloud/triggers/test_kubernetes_engine.py index 076ce1837102..d1f1dd519c57 100644 --- a/providers/tests/google/cloud/triggers/test_kubernetes_engine.py +++ b/providers/tests/google/cloud/triggers/test_kubernetes_engine.py @@ -45,7 +45,6 @@ TRIGGER_GKE_POD_PATH = GKE_TRIGGERS_PATH + ".GKEStartPodTrigger" TRIGGER_GKE_JOB_PATH = GKE_TRIGGERS_PATH + ".GKEJobTrigger" TRIGGER_KUB_POD_PATH = "airflow.providers.cncf.kubernetes.triggers.pod.KubernetesPodTrigger" -HOOK_PATH = "airflow.providers.google.cloud.hooks.kubernetes_engine.GKEPodAsyncHook" POD_NAME = "test-pod-name" JOB_NAME = "test-job-name" NAMESPACE = "default" diff --git a/providers/tests/google/cloud/utils/base_gcp_mock.py b/providers/tests/google/cloud/utils/base_gcp_mock.py index c4295a3191be..a0b556e55a94 100644 --- a/providers/tests/google/cloud/utils/base_gcp_mock.py +++ b/providers/tests/google/cloud/utils/base_gcp_mock.py @@ -30,7 +30,6 @@ def mock_base_gcp_hook_default_project_id( self, gcp_conn_id="google_cloud_default", impersonation_chain=None, - delegate_to=None, ): self.standard_extras_list = {"project": GCP_PROJECT_ID_HOOK_UNIT_TEST} self._conn = gcp_conn_id @@ -39,14 +38,12 @@ def mock_base_gcp_hook_default_project_id( self._conn = None self._cached_credentials = None self._cached_project_id = None - self.delegate_to = delegate_to def mock_base_gcp_hook_no_default_project_id( self, gcp_conn_id="google_cloud_default", impersonation_chain=None, - delegate_to=None, ): self.standard_extras_list = {} self._conn = gcp_conn_id @@ -55,7 +52,6 @@ def mock_base_gcp_hook_no_default_project_id( self._conn = None self._cached_credentials = None self._cached_project_id = None - self.delegate_to = delegate_to if os.environ.get("_AIRFLOW_SKIP_DB_TESTS") == "true": diff --git a/providers/tests/google/common/hooks/test_base_google.py b/providers/tests/google/common/hooks/test_base_google.py index 04cc395f4781..20ec029a9475 100644 --- a/providers/tests/google/common/hooks/test_base_google.py +++ b/providers/tests/google/common/hooks/test_base_google.py @@ -410,7 +410,6 @@ def test_get_credentials_and_project_id_with_default_auth(self, mock_get_creds_a key_secret_name=None, key_secret_project_id=None, scopes=self.instance.scopes, - delegate_to=None, target_principal=None, delegates=None, is_anonymous=None, @@ -452,7 +451,6 @@ def test_get_credentials_and_project_id_with_service_account_file(self, mock_get key_secret_name=None, key_secret_project_id=None, scopes=self.instance.scopes, - delegate_to=None, target_principal=None, delegates=None, is_anonymous=None, @@ -487,7 +485,6 @@ def test_get_credentials_and_project_id_with_service_account_info(self, mock_get key_secret_name=None, key_secret_project_id=None, scopes=self.instance.scopes, - delegate_to=None, target_principal=None, delegates=None, is_anonymous=None, @@ -503,7 +500,6 @@ def test_get_credentials_and_project_id_with_default_auth_and_delegate(self, moc mock_credentials = mock.MagicMock() mock_get_creds_and_proj_id.return_value = (mock_credentials, "PROJECT_ID") self.instance.extras = {} - self.instance.delegate_to = "USER" result = self.instance.get_credentials_and_project_id() mock_get_creds_and_proj_id.assert_called_once_with( key_path=None, @@ -512,7 +508,6 @@ def test_get_credentials_and_project_id_with_default_auth_and_delegate(self, moc key_secret_name=None, key_secret_project_id=None, scopes=self.instance.scopes, - delegate_to="USER", target_principal=None, delegates=None, is_anonymous=None, @@ -523,23 +518,6 @@ def test_get_credentials_and_project_id_with_default_auth_and_delegate(self, moc ) assert (mock_credentials, "PROJECT_ID") == result - @mock.patch("google.auth.default") - def test_get_credentials_and_project_id_with_default_auth_and_unsupported_delegate( - self, mock_auth_default - ): - self.instance.delegate_to = "TEST_DELEGATE_TO" - mock_credentials = mock.MagicMock(spec=google.auth.compute_engine.Credentials) - mock_auth_default.return_value = (mock_credentials, "PROJECT_ID") - - with pytest.raises( - AirflowException, - match=re.escape( - "The `delegate_to` parameter cannot be used here as the current authentication method " - "does not support account impersonate. Please use service-account for authorization." - ), - ): - self.instance.get_credentials_and_project_id() - @mock.patch(MODULE_NAME + ".get_credentials_and_project_id", return_value=("CREDENTIALS", "PROJECT_ID")) def test_get_credentials_and_project_id_with_default_auth_and_overridden_project_id( self, mock_get_creds_and_proj_id @@ -553,7 +531,6 @@ def test_get_credentials_and_project_id_with_default_auth_and_overridden_project key_secret_name=None, key_secret_project_id=None, scopes=self.instance.scopes, - delegate_to=None, target_principal=None, delegates=None, is_anonymous=None, @@ -593,7 +570,6 @@ def test_get_credentials_and_project_id_with_is_anonymous(self, mock_get_creds_a key_secret_name=None, key_secret_project_id=None, scopes=self.instance.scopes, - delegate_to=None, target_principal=None, delegates=None, is_anonymous=True, @@ -805,7 +781,6 @@ def test_get_credentials_and_project_id_with_impersonation_chain( key_secret_name=None, key_secret_project_id=None, scopes=self.instance.scopes, - delegate_to=None, target_principal=target_principal, delegates=delegates, is_anonymous=None, diff --git a/providers/tests/google/marketing_platform/hooks/test_analytics.py b/providers/tests/google/marketing_platform/hooks/test_analytics.py deleted file mode 100644 index 9944e2a9947a..000000000000 --- a/providers/tests/google/marketing_platform/hooks/test_analytics.py +++ /dev/null @@ -1,191 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import warnings -from unittest import mock - -import pytest - -from airflow.exceptions import AirflowProviderDeprecationWarning -from airflow.providers.google.marketing_platform.hooks.analytics import GoogleAnalyticsHook - -from providers.tests.google.cloud.utils.base_gcp_mock import mock_base_gcp_hook_default_project_id - -WEB_PROPERTY_AD_WORDS_LINK_ID = "AAIIRRFFLLOOWW" -WEB_PROPERTY_ID = "web_property_id" -ACCOUNT_ID = "the_knight_who_says_ni!" -DATA_SOURCE = "Monthy Python" -API_VERSION = "v3" -GCP_CONN_ID = "test_gcp_conn_id" -DELEGATE_TO = "TEST_DELEGATE_TO" -IMPERSONATION_CHAIN = ["ACCOUNT_1", "ACCOUNT_2", "ACCOUNT_3"] - - -class TestGoogleAnalyticsHook: - def setup_method(self): - with ( - mock.patch( - "airflow.providers.google.common.hooks.base_google.GoogleBaseHook.__init__", - new=mock_base_gcp_hook_default_project_id, - ), - warnings.catch_warnings(), - ): - warnings.simplefilter("ignore", AirflowProviderDeprecationWarning) - self.hook = GoogleAnalyticsHook(API_VERSION, GCP_CONN_ID) - - @mock.patch("airflow.providers.google.common.hooks.base_google.GoogleBaseHook.__init__") - def test_init(self, mock_base_init): - with pytest.warns(AirflowProviderDeprecationWarning): - hook = GoogleAnalyticsHook( - API_VERSION, - GCP_CONN_ID, - delegate_to=DELEGATE_TO, - impersonation_chain=IMPERSONATION_CHAIN, - ) - mock_base_init.assert_called_once_with( - GCP_CONN_ID, - delegate_to=DELEGATE_TO, - impersonation_chain=IMPERSONATION_CHAIN, - ) - assert hook.api_version == API_VERSION - - @mock.patch("airflow.providers.google.marketing_platform.hooks.analytics.GoogleAnalyticsHook._authorize") - @mock.patch("airflow.providers.google.marketing_platform.hooks.analytics.build") - def test_gen_conn(self, mock_build, mock_authorize): - result = self.hook.get_conn() - mock_build.assert_called_once_with( - "analytics", - API_VERSION, - http=mock_authorize.return_value, - cache_discovery=False, - ) - assert mock_build.return_value == result - - @mock.patch("airflow.providers.google.marketing_platform.hooks.analytics.GoogleAnalyticsHook.get_conn") - def test_list_accounts(self, get_conn_mock): - mock_accounts = get_conn_mock.return_value.management.return_value.accounts - mock_list = mock_accounts.return_value.list - mock_execute = mock_list.return_value.execute - mock_execute.return_value = {"items": ["a", "b"], "totalResults": 2} - list_accounts = self.hook.list_accounts() - assert list_accounts == ["a", "b"] - - @mock.patch("airflow.providers.google.marketing_platform.hooks.analytics.GoogleAnalyticsHook.get_conn") - def test_list_accounts_for_multiple_pages(self, get_conn_mock): - mock_accounts = get_conn_mock.return_value.management.return_value.accounts - mock_list = mock_accounts.return_value.list - mock_execute = mock_list.return_value.execute - mock_execute.side_effect = [ - {"items": ["a"], "totalResults": 2}, - {"items": ["b"], "totalResults": 2}, - ] - list_accounts = self.hook.list_accounts() - assert list_accounts == ["a", "b"] - - @mock.patch("airflow.providers.google.marketing_platform.hooks.analytics.GoogleAnalyticsHook.get_conn") - def test_get_ad_words_links_call(self, get_conn_mock): - num_retries = 5 - self.hook.get_ad_words_link( - account_id=ACCOUNT_ID, - web_property_id=WEB_PROPERTY_ID, - web_property_ad_words_link_id=WEB_PROPERTY_AD_WORDS_LINK_ID, - ) - - get_conn_mock.return_value.management.return_value.webPropertyAdWordsLinks.return_value.get.return_value.execute.assert_called_once_with( - num_retries=num_retries - ) - - get_conn_mock.return_value.management.return_value.webPropertyAdWordsLinks.return_value.get.assert_called_once_with( - accountId=ACCOUNT_ID, - webPropertyId=WEB_PROPERTY_ID, - webPropertyAdWordsLinkId=WEB_PROPERTY_AD_WORDS_LINK_ID, - ) - - @mock.patch("airflow.providers.google.marketing_platform.hooks.analytics.GoogleAnalyticsHook.get_conn") - def test_list_ad_words_links(self, get_conn_mock): - mock_ads_links = get_conn_mock.return_value.management.return_value.webPropertyAdWordsLinks - mock_list = mock_ads_links.return_value.list - mock_execute = mock_list.return_value.execute - mock_execute.return_value = {"items": ["a", "b"], "totalResults": 2} - list_ads_links = self.hook.list_ad_words_links(account_id=ACCOUNT_ID, web_property_id=WEB_PROPERTY_ID) - assert list_ads_links == ["a", "b"] - - @mock.patch("airflow.providers.google.marketing_platform.hooks.analytics.GoogleAnalyticsHook.get_conn") - def test_list_ad_words_links_for_multiple_pages(self, get_conn_mock): - mock_ads_links = get_conn_mock.return_value.management.return_value.webPropertyAdWordsLinks - mock_list = mock_ads_links.return_value.list - mock_execute = mock_list.return_value.execute - mock_execute.side_effect = [ - {"items": ["a"], "totalResults": 2}, - {"items": ["b"], "totalResults": 2}, - ] - list_ads_links = self.hook.list_ad_words_links(account_id=ACCOUNT_ID, web_property_id=WEB_PROPERTY_ID) - assert list_ads_links == ["a", "b"] - - @mock.patch("airflow.providers.google.marketing_platform.hooks.analytics.GoogleAnalyticsHook.get_conn") - @mock.patch("airflow.providers.google.marketing_platform.hooks.analytics.MediaFileUpload") - def test_upload_data(self, media_mock, get_conn_mock): - temp_name = "temp/file" - self.hook.upload_data( - file_location=temp_name, - account_id=ACCOUNT_ID, - web_property_id=WEB_PROPERTY_ID, - custom_data_source_id=DATA_SOURCE, - resumable_upload=True, - ) - - media_mock.assert_called_once_with(temp_name, mimetype="application/octet-stream", resumable=True) - - get_conn_mock.return_value.management.return_value.uploads.return_value.uploadData.assert_called_once_with( - accountId=ACCOUNT_ID, - webPropertyId=WEB_PROPERTY_ID, - customDataSourceId=DATA_SOURCE, - media_body=media_mock.return_value, - ) - - @mock.patch("airflow.providers.google.marketing_platform.hooks.analytics.GoogleAnalyticsHook.get_conn") - def test_delete_upload_data(self, get_conn_mock): - body = {"key": "temp/file"} - self.hook.delete_upload_data( - account_id=ACCOUNT_ID, - web_property_id=WEB_PROPERTY_ID, - custom_data_source_id=DATA_SOURCE, - delete_request_body=body, - ) - - get_conn_mock.return_value.management.return_value.uploads.return_value.deleteUploadData.assert_called_once_with( - accountId=ACCOUNT_ID, - webPropertyId=WEB_PROPERTY_ID, - customDataSourceId=DATA_SOURCE, - body=body, - ) - - @mock.patch("airflow.providers.google.marketing_platform.hooks.analytics.GoogleAnalyticsHook.get_conn") - def test_list_upload(self, get_conn_mock): - uploads = get_conn_mock.return_value.management.return_value.uploads.return_value - uploads.list.return_value.execute.return_value = { - "items": ["a", "b"], - "totalResults": 2, - } - result = self.hook.list_uploads( - account_id=ACCOUNT_ID, - web_property_id=WEB_PROPERTY_ID, - custom_data_source_id=DATA_SOURCE, - ) - assert result == ["a", "b"] diff --git a/providers/tests/google/marketing_platform/operators/test_analytics.py b/providers/tests/google/marketing_platform/operators/test_analytics.py deleted file mode 100644 index 5ef8fa7fddba..000000000000 --- a/providers/tests/google/marketing_platform/operators/test_analytics.py +++ /dev/null @@ -1,273 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -from tempfile import NamedTemporaryFile -from unittest import mock - -import pytest - -from airflow.exceptions import AirflowProviderDeprecationWarning -from airflow.providers.google.marketing_platform.operators.analytics import ( - GoogleAnalyticsDataImportUploadOperator, - GoogleAnalyticsDeletePreviousDataUploadsOperator, - GoogleAnalyticsGetAdsLinkOperator, - GoogleAnalyticsListAccountsOperator, - GoogleAnalyticsModifyFileHeadersDataImportOperator, - GoogleAnalyticsRetrieveAdsLinksListOperator, -) - -WEB_PROPERTY_AD_WORDS_LINK_ID = "AAIIRRFFLLOOWW" -WEB_PROPERTY_ID = "web_property_id" -ACCOUNT_ID = "the_knight_who_says_ni!" -DATA_SOURCE = "Monthy Python" -API_VERSION = "v3" -GCP_CONN_ID = "google_cloud_default" -IMPERSONATION_CHAIN = ["ACCOUNT_1", "ACCOUNT_2", "ACCOUNT_3"] -BUCKET = "gs://bucket" -BUCKET_OBJECT_NAME = "file.csv" - - -class TestGoogleAnalyticsListAccountsOperator: - @mock.patch("airflow.providers.google.marketing_platform.operators.analytics.GoogleAnalyticsHook") - def test_execute(self, hook_mock): - with pytest.warns(AirflowProviderDeprecationWarning): - op = GoogleAnalyticsListAccountsOperator( - api_version=API_VERSION, - gcp_conn_id=GCP_CONN_ID, - task_id="test_task", - impersonation_chain=IMPERSONATION_CHAIN, - ) - op.execute(context=None) - hook_mock.assert_called_once() - hook_mock.return_value.list_accounts.assert_called_once() - - -class TestGoogleAnalyticsRetrieveAdsLinksListOperator: - @mock.patch("airflow.providers.google.marketing_platform.operators.analytics.GoogleAnalyticsHook") - def test_execute(self, hook_mock): - with pytest.warns(AirflowProviderDeprecationWarning): - op = GoogleAnalyticsRetrieveAdsLinksListOperator( - account_id=ACCOUNT_ID, - web_property_id=WEB_PROPERTY_ID, - api_version=API_VERSION, - gcp_conn_id=GCP_CONN_ID, - task_id="test_task", - impersonation_chain=IMPERSONATION_CHAIN, - ) - op.execute(context=None) - hook_mock.assert_called_once() - hook_mock.return_value.list_ad_words_links.assert_called_once() - hook_mock.assert_called_once_with( - gcp_conn_id=GCP_CONN_ID, - api_version=API_VERSION, - impersonation_chain=IMPERSONATION_CHAIN, - ) - hook_mock.return_value.list_ad_words_links.assert_called_once_with( - account_id=ACCOUNT_ID, web_property_id=WEB_PROPERTY_ID - ) - - -class TestGoogleAnalyticsGetAdsLinkOperator: - @mock.patch("airflow.providers.google.marketing_platform.operators.analytics.GoogleAnalyticsHook") - def test_execute(self, hook_mock): - with pytest.warns(AirflowProviderDeprecationWarning): - op = GoogleAnalyticsGetAdsLinkOperator( - account_id=ACCOUNT_ID, - web_property_id=WEB_PROPERTY_ID, - web_property_ad_words_link_id=WEB_PROPERTY_AD_WORDS_LINK_ID, - api_version=API_VERSION, - gcp_conn_id=GCP_CONN_ID, - task_id="test_task", - impersonation_chain=IMPERSONATION_CHAIN, - ) - op.execute(context=None) - hook_mock.assert_called_once() - hook_mock.return_value.get_ad_words_link.assert_called_once() - hook_mock.assert_called_once_with( - gcp_conn_id=GCP_CONN_ID, - api_version=API_VERSION, - impersonation_chain=IMPERSONATION_CHAIN, - ) - hook_mock.return_value.get_ad_words_link.assert_called_once_with( - account_id=ACCOUNT_ID, - web_property_id=WEB_PROPERTY_ID, - web_property_ad_words_link_id=WEB_PROPERTY_AD_WORDS_LINK_ID, - ) - - -class TestGoogleAnalyticsDataImportUploadOperator: - @mock.patch("airflow.providers.google.marketing_platform.operators.analytics.GoogleAnalyticsHook") - @mock.patch("airflow.providers.google.marketing_platform.operators.analytics.GCSHook") - @mock.patch("airflow.providers.google.marketing_platform.operators.analytics.NamedTemporaryFile") - def test_execute(self, mock_tempfile, gcs_hook_mock, ga_hook_mock): - filename = "file/" - mock_tempfile.return_value.__enter__.return_value.name = filename - - with pytest.warns(AirflowProviderDeprecationWarning): - op = GoogleAnalyticsDataImportUploadOperator( - account_id=ACCOUNT_ID, - web_property_id=WEB_PROPERTY_ID, - storage_bucket=BUCKET, - storage_name_object=BUCKET_OBJECT_NAME, - custom_data_source_id=DATA_SOURCE, - api_version=API_VERSION, - gcp_conn_id=GCP_CONN_ID, - task_id="test_task", - impersonation_chain=IMPERSONATION_CHAIN, - ) - op.execute(context=None) - - gcs_hook_mock.assert_called_once_with( - gcp_conn_id=GCP_CONN_ID, - delegate_to=None, - impersonation_chain=IMPERSONATION_CHAIN, - ) - ga_hook_mock.assert_called_once_with( - gcp_conn_id=GCP_CONN_ID, - delegate_to=None, - api_version=API_VERSION, - impersonation_chain=IMPERSONATION_CHAIN, - ) - - gcs_hook_mock.return_value.download.assert_called_once_with( - bucket_name=BUCKET, object_name=BUCKET_OBJECT_NAME, filename=filename - ) - - ga_hook_mock.return_value.upload_data.assert_called_once_with( - filename, ACCOUNT_ID, WEB_PROPERTY_ID, DATA_SOURCE, False - ) - - -class TestGoogleAnalyticsDeletePreviousDataUploadsOperator: - @mock.patch("airflow.providers.google.marketing_platform.operators.analytics.GoogleAnalyticsHook") - def test_execute(self, mock_hook): - mock_hook.return_value.list_uploads.return_value = [ - {"id": 1}, - {"id": 2}, - {"id": 3}, - ] - with pytest.warns(AirflowProviderDeprecationWarning): - op = GoogleAnalyticsDeletePreviousDataUploadsOperator( - account_id=ACCOUNT_ID, - web_property_id=WEB_PROPERTY_ID, - custom_data_source_id=DATA_SOURCE, - api_version=API_VERSION, - gcp_conn_id=GCP_CONN_ID, - task_id="test_task", - ) - op.execute(context=None) - mock_hook.assert_called_once_with( - gcp_conn_id=GCP_CONN_ID, - delegate_to=None, - api_version=API_VERSION, - impersonation_chain=None, - ) - - mock_hook.return_value.list_uploads.assert_called_once_with( - account_id=ACCOUNT_ID, - web_property_id=WEB_PROPERTY_ID, - custom_data_source_id=DATA_SOURCE, - ) - - mock_hook.return_value.delete_upload_data.assert_called_once_with( - ACCOUNT_ID, - WEB_PROPERTY_ID, - DATA_SOURCE, - {"customDataImportUids": [1, 2, 3]}, - ) - - -class TestGoogleAnalyticsModifyFileHeadersDataImportOperator: - def test_modify_column_headers(self): - op = GoogleAnalyticsModifyFileHeadersDataImportOperator( - storage_bucket=BUCKET, - storage_name_object=BUCKET_OBJECT_NAME, - gcp_conn_id=GCP_CONN_ID, - task_id="test_task", - ) - - data = """pagePath,dimension1 -how_to_make_pizza,1 -how_to_make_doughnuts,2 -""" - - # No modification - expected_data = """ga:pagePath,ga:dimension1 -how_to_make_pizza,1 -how_to_make_doughnuts,2 -""" - - with NamedTemporaryFile("w+") as tmp: - tmp.write(data) - tmp.flush() - - op._modify_column_headers(tmp.name, {}) - - with open(tmp.name) as f: - assert expected_data == f.read() - - # with modification - expected_data = """ga:brrr,ga:dimension1 -how_to_make_pizza,1 -how_to_make_doughnuts,2 -""" - with NamedTemporaryFile("w+") as tmp: - tmp.write(data) - tmp.flush() - - op._modify_column_headers(tmp.name, {"pagePath": "brrr"}) - - with open(tmp.name) as f: - assert expected_data == f.read() - - @mock.patch("airflow.providers.google.marketing_platform.operators.analytics.GCSHook") - @mock.patch( - "airflow.providers.google.marketing_platform.operators." - "analytics.GoogleAnalyticsModifyFileHeadersDataImportOperator._modify_column_headers" - ) - @mock.patch("airflow.providers.google.marketing_platform.operators.analytics.NamedTemporaryFile") - def test_execute(self, mock_tempfile, mock_modify, mock_hook): - mapping = {"a": "b"} - filename = "file/" - mock_tempfile.return_value.__enter__.return_value.name = filename - - op = GoogleAnalyticsModifyFileHeadersDataImportOperator( - storage_bucket=BUCKET, - storage_name_object=BUCKET_OBJECT_NAME, - custom_dimension_header_mapping=mapping, - gcp_conn_id=GCP_CONN_ID, - task_id="test_task", - impersonation_chain=IMPERSONATION_CHAIN, - ) - op.execute(context=None) - - mock_hook.assert_called_once_with( - gcp_conn_id=GCP_CONN_ID, - delegate_to=None, - impersonation_chain=IMPERSONATION_CHAIN, - ) - - mock_hook.return_value.download.assert_called_once_with( - bucket_name=BUCKET, object_name=BUCKET_OBJECT_NAME, filename=filename - ) - - mock_modify.assert_called_once_with( - tmp_file_location=filename, custom_dimension_header_mapping=mapping - ) - - mock_hook.return_value.upload(bucket_name=BUCKET, object_name=BUCKET_OBJECT_NAME, filename=filename) diff --git a/providers/tests/google/marketing_platform/operators/test_campaign_manager.py b/providers/tests/google/marketing_platform/operators/test_campaign_manager.py index 57d2b58ddd6d..a366e45561e4 100644 --- a/providers/tests/google/marketing_platform/operators/test_campaign_manager.py +++ b/providers/tests/google/marketing_platform/operators/test_campaign_manager.py @@ -77,7 +77,6 @@ def test_execute(self, mock_base_op, hook_mock): op.execute(context=None) hook_mock.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, - delegate_to=None, api_version=API_VERSION, impersonation_chain=None, ) @@ -134,7 +133,6 @@ def test_execute( op.execute(context=None) hook_mock.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, - delegate_to=None, api_version=API_VERSION, impersonation_chain=None, ) @@ -143,7 +141,6 @@ def test_execute( ) gcs_hook_mock.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, - delegate_to=None, impersonation_chain=None, ) gcs_hook_mock.return_value.upload.assert_called_once_with( @@ -239,7 +236,6 @@ def test_execute(self, xcom_mock, mock_base_op, hook_mock): op.execute(context=None) hook_mock.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, - delegate_to=None, api_version=API_VERSION, impersonation_chain=None, ) @@ -287,7 +283,6 @@ def test_execute(self, xcom_mock, mock_base_op, hook_mock): op.execute(context=None) hook_mock.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, - delegate_to=None, api_version=API_VERSION, impersonation_chain=None, ) diff --git a/providers/tests/google/marketing_platform/operators/test_display_video.py b/providers/tests/google/marketing_platform/operators/test_display_video.py index 5f5282c7ab5d..8c50299052d1 100644 --- a/providers/tests/google/marketing_platform/operators/test_display_video.py +++ b/providers/tests/google/marketing_platform/operators/test_display_video.py @@ -40,7 +40,6 @@ API_VERSION = "v2" GCP_CONN_ID = "google_cloud_default" -DELEGATE_TO: str | None = None IMPERSONATION_CHAIN = ["ACCOUNT_1", "ACCOUNT_2", "ACCOUNT_3"] DEFAULT_DATE = timezone.datetime(2021, 1, 1) @@ -62,7 +61,6 @@ def test_execute(self, hook_mock): op.execute(context=None) hook_mock.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, - delegate_to=None, api_version=API_VERSION, impersonation_chain=None, ) @@ -126,7 +124,6 @@ def test_execute( op.execute(context=None) mock_hook.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, - delegate_to=None, api_version="v2", impersonation_chain=None, ) @@ -134,7 +131,6 @@ def test_execute( mock_gcs_hook.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, - delegate_to=None, impersonation_chain=None, ) mock_gcs_hook.return_value.upload.assert_called_once_with( @@ -231,7 +227,6 @@ def test_execute(self, hook_mock, mock_xcom): op.execute(context=None) hook_mock.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, - delegate_to=None, api_version=API_VERSION, impersonation_chain=None, ) @@ -264,7 +259,6 @@ def test_execute(self, mock_temp, gcs_hook_mock, hook_mock): gzip=gzip, api_version=API_VERSION, gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, task_id="test_task", impersonation_chain=IMPERSONATION_CHAIN, ) @@ -281,13 +275,11 @@ def test_execute(self, mock_temp, gcs_hook_mock, hook_mock): gcs_hook_mock.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, ) hook_mock.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, api_version=API_VERSION, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, ) hook_mock.return_value.download_line_items.assert_called_once_with(request_body=request_body) @@ -315,13 +307,11 @@ def test_execute(self, gcs_hook_mock, hook_mock, mock_tempfile): hook_mock.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, api_version=API_VERSION, - delegate_to=DELEGATE_TO, impersonation_chain=None, ) gcs_hook_mock.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=None, ) @@ -364,7 +354,6 @@ def test_execute(self, mock_temp, gcs_mock_hook, mock_hook): mock_hook.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, api_version=API_VERSION, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, ) @@ -390,7 +379,6 @@ def test_execute(self, mock_temp, gcs_mock_hook, mock_hook): gcs_mock_hook.assert_called_once() gcs_mock_hook.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, ) @@ -431,7 +419,6 @@ def test_execute(self, mock_hook, xcom_mock): mock_hook.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, api_version=API_VERSION, - delegate_to=DELEGATE_TO, impersonation_chain=None, ) @@ -457,7 +444,6 @@ def test_execute(self, hook_mock, xcom_mock): op.execute(context=None) hook_mock.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, - delegate_to=None, api_version="v2", impersonation_chain=None, ) diff --git a/providers/tests/google/marketing_platform/sensors/test_campaign_manager.py b/providers/tests/google/marketing_platform/sensors/test_campaign_manager.py index 3c74a3c8f117..ec501d95302c 100644 --- a/providers/tests/google/marketing_platform/sensors/test_campaign_manager.py +++ b/providers/tests/google/marketing_platform/sensors/test_campaign_manager.py @@ -49,7 +49,6 @@ def test_execute(self, mock_base_op, hook_mock): result = op.poke(context=None) hook_mock.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, - delegate_to=None, api_version=API_VERSION, impersonation_chain=None, ) diff --git a/providers/tests/google/marketing_platform/sensors/test_display_video.py b/providers/tests/google/marketing_platform/sensors/test_display_video.py index 68621e43bf2a..4b5ef852eef2 100644 --- a/providers/tests/google/marketing_platform/sensors/test_display_video.py +++ b/providers/tests/google/marketing_platform/sensors/test_display_video.py @@ -43,7 +43,6 @@ def test_poke(self, mock_base_op, hook_mock): op.poke(context=None) hook_mock.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, - delegate_to=None, api_version="v2", impersonation_chain=None, ) @@ -63,7 +62,6 @@ def test_poke(self, mock_base_op, hook_mock): op.poke(context=None) hook_mock.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, - delegate_to=None, api_version=API_VERSION, impersonation_chain=None, ) diff --git a/providers/tests/google/suite/sensors/test_drive.py b/providers/tests/google/suite/sensors/test_drive.py index c5c3fdc26acb..f77d4964c52c 100644 --- a/providers/tests/google/suite/sensors/test_drive.py +++ b/providers/tests/google/suite/sensors/test_drive.py @@ -26,7 +26,6 @@ TEST_FILE_NAME = os.environ.get("GCP_GDRIVE_DRIVE_ID", "gdrive_to_gcs_file.txt") TEST_DRIVE_ID = os.environ.get("GCP_GDRIVE_TO_GCS_FILE_NAME", "abcd1234") TEST_GCP_CONN_ID = "google_cloud_default" -TEST_DELEGATE_TO = "TEST_DELEGATE_TO" TEST_IMPERSONATION_CHAIN = ["ACCOUNT_1", "ACCOUNT_2", "ACCOUNT_3"] @@ -39,7 +38,6 @@ def test_should_pass_argument_to_hook(self, mock_hook): file_name=TEST_FILE_NAME, drive_id=TEST_DRIVE_ID, gcp_conn_id=TEST_GCP_CONN_ID, - delegate_to=TEST_DELEGATE_TO, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) @@ -49,7 +47,6 @@ def test_should_pass_argument_to_hook(self, mock_hook): assert result mock_hook.assert_called_once_with( - delegate_to=TEST_DELEGATE_TO, gcp_conn_id=TEST_GCP_CONN_ID, impersonation_chain=TEST_IMPERSONATION_CHAIN, ) diff --git a/providers/tests/google/suite/transfers/test_gcs_to_gdrive.py b/providers/tests/google/suite/transfers/test_gcs_to_gdrive.py index 3e81e166bda5..8d60f3ebe203 100644 --- a/providers/tests/google/suite/transfers/test_gcs_to_gdrive.py +++ b/providers/tests/google/suite/transfers/test_gcs_to_gdrive.py @@ -26,7 +26,6 @@ MODULE = "airflow.providers.google.suite.transfers.gcs_to_gdrive" IMPERSONATION_CHAIN = ["ACCOUNT_1", "ACCOUNT_2", "ACCOUNT_3"] -DELEGATE_TO = "test_account@xxx.zzz" class TestGcsToGDriveOperator: @@ -43,7 +42,6 @@ def test_should_copy_single_file(self, mock_named_temporary_file, mock_gdrive, m source_object="sales/sales-2017/january.avro", destination_object="copied_sales/2017/january-backup.avro", impersonation_chain=None, - delegate_to=DELEGATE_TO, ) task.execute(mock.MagicMock()) @@ -63,7 +61,6 @@ def test_should_copy_single_file(self, mock_named_temporary_file, mock_gdrive, m mock_gdrive.assert_has_calls( [ mock.call( - delegate_to=DELEGATE_TO, gcp_conn_id="google_cloud_default", impersonation_chain=None, ), @@ -88,7 +85,6 @@ def test_should_copy_single_file_with_folder(self, mock_named_temporary_file, mo source_object="sales/sales-2017/january.avro", destination_object="copied_sales/2017/january-backup.avro", destination_folder_id="aAopls6bE4tUllZVGJvRUU", - delegate_to=DELEGATE_TO, ) task.execute(mock.MagicMock()) @@ -109,7 +105,6 @@ def test_should_copy_single_file_with_folder(self, mock_named_temporary_file, mo [ mock.call( gcp_conn_id="google_cloud_default", - delegate_to=DELEGATE_TO, impersonation_chain=None, ), mock.call().upload_file( @@ -136,7 +131,6 @@ def test_should_copy_files(self, mock_named_temporary_file, mock_gdrive, mock_gc source_object="sales/sales-2017/*.avro", destination_object="copied_sales/2017/", impersonation_chain=IMPERSONATION_CHAIN, - delegate_to=DELEGATE_TO, ) task.execute(mock.MagicMock()) @@ -159,7 +153,6 @@ def test_should_copy_files(self, mock_named_temporary_file, mock_gdrive, mock_gc mock_gdrive.assert_has_calls( [ mock.call( - delegate_to=DELEGATE_TO, gcp_conn_id="google_cloud_default", impersonation_chain=IMPERSONATION_CHAIN, ), @@ -189,7 +182,6 @@ def test_should_move_files(self, mock_named_temporary_file, mock_gdrive, mock_gc source_object="sales/sales-2017/*.avro", move_object=True, impersonation_chain=IMPERSONATION_CHAIN, - delegate_to=DELEGATE_TO, ) task.execute(mock.MagicMock()) @@ -215,7 +207,6 @@ def test_should_move_files(self, mock_named_temporary_file, mock_gdrive, mock_gc mock_gdrive.assert_has_calls( [ mock.call( - delegate_to=DELEGATE_TO, gcp_conn_id="google_cloud_default", impersonation_chain=IMPERSONATION_CHAIN, ), diff --git a/providers/tests/google/suite/transfers/test_gcs_to_sheets.py b/providers/tests/google/suite/transfers/test_gcs_to_sheets.py index 0e624284e5e1..30f48d513737 100644 --- a/providers/tests/google/suite/transfers/test_gcs_to_sheets.py +++ b/providers/tests/google/suite/transfers/test_gcs_to_sheets.py @@ -26,7 +26,6 @@ VALUES = [[1, 2, 3]] BUCKET = "destination_bucket" PATH = "path/to/reports" -DELEGATE_TO = "test_account@xxx.zzz" class TestGCSToGoogleSheets: @@ -48,13 +47,11 @@ def test_execute(self, mock_reader, mock_tempfile, mock_sheet_hook, mock_gcs_hoo object_name=PATH, gcp_conn_id=GCP_CONN_ID, impersonation_chain=IMPERSONATION_CHAIN, - delegate_to=DELEGATE_TO, ) op.execute(None) mock_sheet_hook.assert_called_once_with( gcp_conn_id=GCP_CONN_ID, - delegate_to=DELEGATE_TO, impersonation_chain=IMPERSONATION_CHAIN, ) mock_gcs_hook.assert_called_once_with( diff --git a/providers/tests/google/suite/transfers/test_sql_to_sheets.py b/providers/tests/google/suite/transfers/test_sql_to_sheets.py index 9ecffc5edb2f..c33c9cdf9c2c 100644 --- a/providers/tests/google/suite/transfers/test_sql_to_sheets.py +++ b/providers/tests/google/suite/transfers/test_sql_to_sheets.py @@ -53,7 +53,6 @@ def test_execute(self, mock_sheet_hook): mock_sheet_hook.assert_called_once_with( gcp_conn_id=self.gcp_conn_id, - delegate_to=None, impersonation_chain=None, ) diff --git a/providers/tests/system/google/cloud/gcs/example_gcs_sensor.py b/providers/tests/system/google/cloud/gcs/example_gcs_sensor.py index a07aff849e46..110a86ce42a5 100644 --- a/providers/tests/system/google/cloud/gcs/example_gcs_sensor.py +++ b/providers/tests/system/google/cloud/gcs/example_gcs_sensor.py @@ -28,7 +28,6 @@ from airflow.models.dag import DAG from airflow.providers.google.cloud.operators.gcs import GCSCreateBucketOperator, GCSDeleteBucketOperator from airflow.providers.google.cloud.sensors.gcs import ( - GCSObjectExistenceAsyncSensor, GCSObjectExistenceSensor, GCSObjectsWithPrefixExistenceSensor, GCSObjectUpdateSensor, @@ -142,14 +141,6 @@ def mode_setter(self, value): ) # [END howto_sensor_object_exists_task] - # [START howto_sensor_object_exists_task_async] - gcs_object_exists_async = GCSObjectExistenceAsyncSensor( - bucket=DESTINATION_BUCKET_NAME, - object=FILE_NAME, - task_id="gcs_object_exists_task_async", - ) - # [END howto_sensor_object_exists_task_async] - # [START howto_sensor_object_exists_task_defered] gcs_object_exists_defered = GCSObjectExistenceSensor( bucket=DESTINATION_BUCKET_NAME, object=FILE_NAME, task_id="gcs_object_exists_defered", deferrable=True @@ -185,7 +176,6 @@ def mode_setter(self, value): [ gcs_object_exists, gcs_object_exists_defered, - gcs_object_exists_async, gcs_object_with_prefix_exists, gcs_object_with_prefix_exists_async, ], diff --git a/providers/tests/system/google/cloud/vertex_ai/example_vertex_ai_custom_job.py b/providers/tests/system/google/cloud/vertex_ai/example_vertex_ai_custom_job.py index eb2a080a92a1..22c3510d1023 100644 --- a/providers/tests/system/google/cloud/vertex_ai/example_vertex_ai_custom_job.py +++ b/providers/tests/system/google/cloud/vertex_ai/example_vertex_ai_custom_job.py @@ -171,7 +171,6 @@ def TABULAR_DATASET(bucket_name): dataset_id=tabular_dataset_id, replica_count=REPLICA_COUNT, model_display_name=MODEL_DISPLAY_NAME, - sync=False, region=REGION, project_id=PROJECT_ID, ) @@ -191,7 +190,6 @@ def TABULAR_DATASET(bucket_name): dataset_id=tabular_dataset_id, replica_count=REPLICA_COUNT, model_display_name=f"{MODEL_DISPLAY_NAME}-def", - sync=False, region=REGION, project_id=PROJECT_ID, deferrable=True, diff --git a/providers/tests/system/google/cloud/vertex_ai/example_vertex_ai_hyperparameter_tuning_job.py b/providers/tests/system/google/cloud/vertex_ai/example_vertex_ai_hyperparameter_tuning_job.py index 4fabdbc6ac67..2a3721d0d0a6 100644 --- a/providers/tests/system/google/cloud/vertex_ai/example_vertex_ai_hyperparameter_tuning_job.py +++ b/providers/tests/system/google/cloud/vertex_ai/example_vertex_ai_hyperparameter_tuning_job.py @@ -95,7 +95,6 @@ staging_bucket=STAGING_BUCKET, display_name=DISPLAY_NAME, worker_pool_specs=WORKER_POOL_SPECS, - sync=False, region=REGION, project_id=PROJECT_ID, parameter_spec=PARAM_SPECS, @@ -111,7 +110,6 @@ staging_bucket=STAGING_BUCKET, display_name=DISPLAY_NAME, worker_pool_specs=WORKER_POOL_SPECS, - sync=False, region=REGION, project_id=PROJECT_ID, parameter_spec=PARAM_SPECS, diff --git a/providers/tests/system/google/cloud/vertex_ai/example_vertex_ai_model_service.py b/providers/tests/system/google/cloud/vertex_ai/example_vertex_ai_model_service.py index 1aaeccc9c069..e29cc4ea198a 100644 --- a/providers/tests/system/google/cloud/vertex_ai/example_vertex_ai_model_service.py +++ b/providers/tests/system/google/cloud/vertex_ai/example_vertex_ai_model_service.py @@ -179,7 +179,6 @@ dataset_id=tabular_dataset_id, replica_count=1, model_display_name=MODEL_DISPLAY_NAME, - sync=False, region=REGION, project_id=PROJECT_ID, ) @@ -198,7 +197,6 @@ dataset_id=tabular_dataset_id, replica_count=1, model_display_name=MODEL_DISPLAY_NAME, - sync=False, region=REGION, project_id=PROJECT_ID, ) diff --git a/tests/always/test_project_structure.py b/tests/always/test_project_structure.py index 8651bd968c2a..1960f403b04e 100644 --- a/tests/always/test_project_structure.py +++ b/tests/always/test_project_structure.py @@ -355,13 +355,7 @@ class TestGoogleProviderProjectStructure(ExampleCoverageTest, AssetsCoverageTest "airflow.providers.google.cloud.operators.automl.AutoMLDeployModelOperator", "airflow.providers.google.cloud.operators.datapipeline.CreateDataPipelineOperator", "airflow.providers.google.cloud.operators.datapipeline.RunDataPipelineOperator", - "airflow.providers.google.cloud.operators.dataproc.DataprocSubmitHadoopJobOperator", "airflow.providers.google.cloud.operators.dataproc.DataprocScaleClusterOperator", - "airflow.providers.google.cloud.operators.dataproc.DataprocSubmitSparkJobOperator", - "airflow.providers.google.cloud.operators.dataproc.DataprocSubmitSparkSqlJobOperator", - "airflow.providers.google.cloud.operators.dataproc.DataprocSubmitHiveJobOperator", - "airflow.providers.google.cloud.operators.dataproc.DataprocSubmitPigJobOperator", - "airflow.providers.google.cloud.operators.dataproc.DataprocSubmitPySparkJobOperator", "airflow.providers.google.cloud.operators.mlengine.MLEngineManageModelOperator", "airflow.providers.google.cloud.operators.mlengine.MLEngineManageVersionOperator", "airflow.providers.google.cloud.operators.mlengine.MLEngineCreateModelOperator", @@ -373,24 +367,10 @@ class TestGoogleProviderProjectStructure(ExampleCoverageTest, AssetsCoverageTest "airflow.providers.google.cloud.operators.mlengine.MLEngineSetDefaultVersionOperator", "airflow.providers.google.cloud.operators.mlengine.MLEngineStartBatchPredictionJobOperator", "airflow.providers.google.cloud.operators.mlengine.MLEngineStartTrainingJobOperator", - "airflow.providers.google.cloud.operators.dataflow.DataflowCreateJavaJobOperator", - "airflow.providers.google.cloud.operators.bigquery.BigQueryPatchDatasetOperator", - "airflow.providers.google.cloud.operators.dataflow.DataflowCreatePythonJobOperator", - "airflow.providers.google.cloud.operators.bigquery.BigQueryExecuteQueryOperator", - "airflow.providers.google.cloud.operators.vertex_ai.auto_ml.CreateAutoMLTextTrainingJobOperator", - "airflow.providers.google.cloud.sensors.bigquery.BigQueryTableExistenceAsyncSensor", - "airflow.providers.google.cloud.sensors.bigquery.BigQueryTableExistencePartitionAsyncSensor", - "airflow.providers.google.cloud.sensors.cloud_composer.CloudComposerEnvironmentSensor", "airflow.providers.google.marketing_platform.operators.GoogleDisplayVideo360CreateQueryOperator", "airflow.providers.google.marketing_platform.operators.GoogleDisplayVideo360RunQueryOperator", "airflow.providers.google.marketing_platform.operators.GoogleDisplayVideo360DownloadReportV2Operator", "airflow.providers.google.marketing_platform.sensors.GoogleDisplayVideo360RunQuerySensor", - "airflow.providers.google.marketing_platform.operators.analytics.GoogleAnalyticsDataImportUploadOperator", - "airflow.providers.google.marketing_platform.operators.analytics.GoogleAnalyticsDeletePreviousDataUploadsOperator", - "airflow.providers.google.marketing_platform.operators.analytics.GoogleAnalyticsGetAdsLinkOperator", - "airflow.providers.google.marketing_platform.operators.analytics.GoogleAnalyticsListAccountsOperator", - "airflow.providers.google.marketing_platform.operators.analytics.GoogleAnalyticsModifyFileHeadersDataImportOperator", - "airflow.providers.google.marketing_platform.operators.analytics.GoogleAnalyticsRetrieveAdsLinksListOperator", } BASE_CLASSES = {