From a847bc57ace2c11c105bba739b3bd98238c5c11d Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Tue, 25 Feb 2025 15:02:45 +0100 Subject: [PATCH] Add legacy namespace packages to airflow.providers It seem that vscode cannot recognise cross-provider imports when providers have implicit namespaces for "airflow.providers" and "airflow.providers.nnnn" (for "common" etc.). And what seem to halp is to add legacy namespace `__init__.py` files for those packages. This has no impact on generated packages - because generated packages have "tool.flit.module" set to the right sub-package and flit will not use the __init__.py files when packages are generated. --- docs/conf.py | 8 ++ providers/airbyte/src/airflow/__init__.py | 17 ++++ .../airbyte/src/airflow/providers/__init__.py | 17 ++++ providers/alibaba/src/airflow/__init__.py | 17 ++++ .../alibaba/src/airflow/providers/__init__.py | 17 ++++ providers/amazon/src/airflow/__init__.py | 17 ++++ .../amazon/src/airflow/providers/__init__.py | 17 ++++ providers/apache/beam/src/airflow/__init__.py | 17 ++++ .../beam/src/airflow/providers/__init__.py | 17 ++++ .../src/airflow/providers/apache/__init__.py | 17 ++++ .../apache/cassandra/src/airflow/__init__.py | 17 ++++ .../src/airflow/providers/__init__.py | 17 ++++ .../src/airflow/providers/apache/__init__.py | 17 ++++ .../apache/cassandra/hooks/cassandra.py | 5 +- .../apache/drill/src/airflow/__init__.py | 17 ++++ .../drill/src/airflow/providers/__init__.py | 17 ++++ .../src/airflow/providers/apache/__init__.py | 17 ++++ .../apache/druid/src/airflow/__init__.py | 17 ++++ .../druid/src/airflow/providers/__init__.py | 17 ++++ .../src/airflow/providers/apache/__init__.py | 17 ++++ .../apache/flink/src/airflow/__init__.py | 17 ++++ .../flink/src/airflow/providers/__init__.py | 17 ++++ .../src/airflow/providers/apache/__init__.py | 17 ++++ providers/apache/hdfs/src/airflow/__init__.py | 17 ++++ .../hdfs/src/airflow/providers/__init__.py | 17 ++++ .../src/airflow/providers/apache/__init__.py | 17 ++++ .../providers/apache/hdfs/hooks/webhdfs.py | 2 +- .../providers/apache/hdfs/sensors/web_hdfs.py | 3 +- providers/apache/hive/src/airflow/__init__.py | 17 ++++ .../hive/src/airflow/providers/__init__.py | 17 ++++ .../src/airflow/providers/apache/__init__.py | 17 ++++ .../apache/iceberg/src/airflow/__init__.py | 17 ++++ .../iceberg/src/airflow/providers/__init__.py | 17 ++++ .../src/airflow/providers/apache/__init__.py | 17 ++++ .../apache/impala/src/airflow/__init__.py | 17 ++++ .../impala/src/airflow/providers/__init__.py | 17 ++++ .../src/airflow/providers/apache/__init__.py | 17 ++++ .../providers/apache/impala/hooks/impala.py | 3 +- .../apache/kafka/src/airflow/__init__.py | 17 ++++ .../kafka/src/airflow/providers/__init__.py | 17 ++++ .../src/airflow/providers/apache/__init__.py | 17 ++++ .../apache/kylin/src/airflow/__init__.py | 17 ++++ .../kylin/src/airflow/providers/__init__.py | 17 ++++ .../src/airflow/providers/apache/__init__.py | 17 ++++ providers/apache/livy/src/airflow/__init__.py | 17 ++++ .../livy/src/airflow/providers/__init__.py | 17 ++++ .../src/airflow/providers/apache/__init__.py | 17 ++++ providers/apache/pig/src/airflow/__init__.py | 17 ++++ .../pig/src/airflow/providers/__init__.py | 17 ++++ .../src/airflow/providers/apache/__init__.py | 17 ++++ .../apache/pinot/src/airflow/__init__.py | 17 ++++ .../pinot/src/airflow/providers/__init__.py | 17 ++++ .../src/airflow/providers/apache/__init__.py | 17 ++++ .../apache/spark/src/airflow/__init__.py | 17 ++++ .../spark/src/airflow/providers/__init__.py | 17 ++++ .../src/airflow/providers/apache/__init__.py | 17 ++++ providers/apprise/src/airflow/__init__.py | 17 ++++ .../apprise/src/airflow/providers/__init__.py | 17 ++++ .../providers/apprise/hooks/apprise.py | 3 +- .../apprise/notifications/apprise.py | 3 +- providers/arangodb/src/airflow/__init__.py | 17 ++++ .../src/airflow/providers/__init__.py | 17 ++++ providers/asana/src/airflow/__init__.py | 17 ++++ .../asana/src/airflow/providers/__init__.py | 17 ++++ .../airflow/providers/asana/hooks/asana.py | 3 +- .../atlassian/jira/src/airflow/__init__.py | 17 ++++ .../jira/src/airflow/providers/__init__.py | 17 ++++ .../airflow/providers/atlassian/__init__.py | 17 ++++ providers/celery/src/airflow/__init__.py | 17 ++++ .../celery/src/airflow/providers/__init__.py | 17 ++++ .../providers/celery/cli/celery_command.py | 6 +- .../celery/executors/celery_executor.py | 2 +- .../celery/executors/celery_executor_utils.py | 11 +-- .../providers/celery/sensors/celery_queue.py | 3 +- providers/cloudant/src/airflow/__init__.py | 17 ++++ .../src/airflow/providers/__init__.py | 17 ++++ .../cncf/kubernetes/src/airflow/__init__.py | 17 ++++ .../src/airflow/providers/__init__.py | 17 ++++ .../src/airflow/providers/cncf/__init__.py | 17 ++++ .../backcompat/backwards_compat_converters.py | 3 +- .../providers/cncf/kubernetes/callbacks.py | 3 +- .../cncf/kubernetes/cli/kubernetes_command.py | 7 +- .../cncf/kubernetes/decorators/kubernetes.py | 2 +- .../executors/kubernetes_executor.py | 6 +- .../executors/kubernetes_executor_utils.py | 7 +- .../cncf/kubernetes/hooks/kubernetes.py | 6 +- .../kubernetes/kubernetes_helper_functions.py | 2 +- .../operators/custom_object_launcher.py | 4 +- .../cncf/kubernetes/operators/job.py | 7 +- .../cncf/kubernetes/operators/kueue.py | 3 +- .../cncf/kubernetes/operators/pod.py | 8 +- .../cncf/kubernetes/operators/resource.py | 2 +- .../kubernetes/operators/spark_kubernetes.py | 3 +- .../cncf/kubernetes/pod_generator.py | 4 +- .../resource_convert/env_variable.py | 3 +- .../providers/cncf/kubernetes/secret.py | 3 +- .../kubernetes/sensors/spark_kubernetes.py | 3 +- .../cncf/kubernetes/template_rendering.py | 2 +- .../kubernetes/utils/k8s_resource_iterator.py | 3 +- .../cncf/kubernetes/utils/pod_manager.py | 9 +- providers/cohere/src/airflow/__init__.py | 17 ++++ .../cohere/src/airflow/providers/__init__.py | 17 ++++ .../airflow/providers/cohere/hooks/cohere.py | 3 +- .../common/compat/src/airflow/__init__.py | 17 ++++ .../compat/src/airflow/providers/__init__.py | 17 ++++ .../src/airflow/providers/common/__init__.py | 17 ++++ providers/common/io/src/airflow/__init__.py | 17 ++++ .../io/src/airflow/providers/__init__.py | 17 ++++ .../src/airflow/providers/common/__init__.py | 17 ++++ providers/common/sql/src/airflow/__init__.py | 17 ++++ .../sql/src/airflow/providers/__init__.py | 17 ++++ .../src/airflow/providers/common/__init__.py | 17 ++++ providers/databricks/src/airflow/__init__.py | 17 ++++ .../src/airflow/providers/__init__.py | 17 ++++ .../databricks/hooks/databricks_sql.py | 5 +- .../databricks/operators/databricks_sql.py | 3 +- .../sensors/databricks_partition.py | 3 +- providers/datadog/src/airflow/__init__.py | 17 ++++ .../datadog/src/airflow/providers/__init__.py | 17 ++++ .../providers/datadog/hooks/datadog.py | 3 +- .../providers/datadog/sensors/datadog.py | 3 +- providers/dbt/cloud/src/airflow/__init__.py | 17 ++++ .../cloud/src/airflow/providers/__init__.py | 17 ++++ .../src/airflow/providers/dbt/__init__.py | 17 ++++ providers/dingding/src/airflow/__init__.py | 17 ++++ .../src/airflow/providers/__init__.py | 17 ++++ providers/discord/src/airflow/__init__.py | 17 ++++ .../discord/src/airflow/providers/__init__.py | 17 ++++ providers/docker/src/airflow/__init__.py | 17 ++++ .../docker/src/airflow/providers/__init__.py | 17 ++++ .../airflow/providers/docker/hooks/docker.py | 5 +- .../providers/docker/operators/docker.py | 6 +- .../docker/operators/docker_swarm.py | 5 +- providers/edge/src/airflow/__init__.py | 17 ++++ .../edge/src/airflow/providers/__init__.py | 17 ++++ .../elasticsearch/src/airflow/__init__.py | 17 ++++ .../src/airflow/providers/__init__.py | 17 ++++ .../elasticsearch/hooks/elasticsearch.py | 3 +- .../elasticsearch/log/es_task_handler.py | 8 +- providers/exasol/src/airflow/__init__.py | 17 ++++ .../exasol/src/airflow/providers/__init__.py | 17 ++++ providers/fab/src/airflow/__init__.py | 17 ++++ .../3rd-party-licenses/LICENSES-ui.txt | 89 ------------------- .../fab/src/airflow/providers/__init__.py | 17 ++++ providers/facebook/src/airflow/__init__.py | 17 ++++ .../src/airflow/providers/__init__.py | 17 ++++ providers/ftp/src/airflow/__init__.py | 17 ++++ .../ftp/src/airflow/providers/__init__.py | 17 ++++ providers/github/src/airflow/__init__.py | 17 ++++ .../github/src/airflow/providers/__init__.py | 17 ++++ .../airflow/providers/github/hooks/github.py | 3 +- .../providers/github/operators/github.py | 3 +- .../providers/github/sensors/github.py | 3 +- providers/google/src/airflow/__init__.py | 17 ++++ .../google/src/airflow/providers/__init__.py | 17 ++++ .../airflow/providers/google/ads/hooks/ads.py | 7 +- .../_internal_client/secret_manager_client.py | 5 +- .../providers/google/cloud/hooks/alloy_db.py | 5 +- .../providers/google/cloud/hooks/automl.py | 9 +- .../providers/google/cloud/hooks/bigquery.py | 41 +++++---- .../google/cloud/hooks/bigquery_dts.py | 16 ++-- .../providers/google/cloud/hooks/bigtable.py | 5 +- .../google/cloud/hooks/cloud_batch.py | 7 +- .../google/cloud/hooks/cloud_build.py | 9 +- .../google/cloud/hooks/cloud_composer.py | 7 +- .../google/cloud/hooks/cloud_memorystore.py | 7 +- .../providers/google/cloud/hooks/cloud_run.py | 7 +- .../providers/google/cloud/hooks/cloud_sql.py | 2 +- .../hooks/cloud_storage_transfer_service.py | 17 ++-- .../providers/google/cloud/hooks/compute.py | 6 +- .../google/cloud/hooks/datacatalog.py | 7 +- .../providers/google/cloud/hooks/dataflow.py | 24 ++--- .../providers/google/cloud/hooks/dataform.py | 5 +- .../google/cloud/hooks/datafusion.py | 4 +- .../providers/google/cloud/hooks/dataplex.py | 18 ++-- .../providers/google/cloud/hooks/dataproc.py | 9 +- .../google/cloud/hooks/dataproc_metastore.py | 7 +- .../providers/google/cloud/hooks/dlp.py | 7 +- .../providers/google/cloud/hooks/gcs.py | 13 ++- .../providers/google/cloud/hooks/kms.py | 5 +- .../google/cloud/hooks/kubernetes_engine.py | 16 ++-- .../google/cloud/hooks/life_sciences.py | 2 +- .../google/cloud/hooks/managed_kafka.py | 5 +- .../google/cloud/hooks/natural_language.py | 5 +- .../providers/google/cloud/hooks/os_login.py | 5 +- .../providers/google/cloud/hooks/pubsub.py | 12 +-- .../google/cloud/hooks/secret_manager.py | 5 +- .../providers/google/cloud/hooks/spanner.py | 4 +- .../google/cloud/hooks/speech_to_text.py | 5 +- .../google/cloud/hooks/stackdriver.py | 8 +- .../providers/google/cloud/hooks/tasks.py | 7 +- .../google/cloud/hooks/text_to_speech.py | 5 +- .../providers/google/cloud/hooks/translate.py | 10 +-- .../google/cloud/hooks/vertex_ai/auto_ml.py | 7 +- .../hooks/vertex_ai/batch_prediction_job.py | 7 +- .../cloud/hooks/vertex_ai/custom_job.py | 9 +- .../google/cloud/hooks/vertex_ai/dataset.py | 7 +- .../cloud/hooks/vertex_ai/endpoint_service.py | 5 +- .../cloud/hooks/vertex_ai/feature_store.py | 7 +- .../vertex_ai/hyperparameter_tuning_job.py | 7 +- .../cloud/hooks/vertex_ai/model_service.py | 5 +- .../cloud/hooks/vertex_ai/pipeline_job.py | 7 +- .../hooks/vertex_ai/prediction_service.py | 5 +- .../google/cloud/hooks/video_intelligence.py | 5 +- .../providers/google/cloud/hooks/vision.py | 7 +- .../providers/google/cloud/hooks/workflows.py | 5 +- .../google/cloud/links/kubernetes_engine.py | 3 +- .../google/cloud/log/gcs_task_handler.py | 6 +- .../cloud/log/stackdriver_task_handler.py | 14 +-- .../google/cloud/openlineage/utils.py | 6 +- .../google/cloud/operators/alloy_db.py | 11 +-- .../google/cloud/operators/automl.py | 20 +++-- .../google/cloud/operators/bigquery.py | 12 +-- .../google/cloud/operators/bigquery_dts.py | 18 ++-- .../google/cloud/operators/bigtable.py | 4 +- .../google/cloud/operators/cloud_base.py | 3 +- .../google/cloud/operators/cloud_batch.py | 6 +- .../google/cloud/operators/cloud_build.py | 8 +- .../google/cloud/operators/cloud_composer.py | 12 +-- .../cloud/operators/cloud_memorystore.py | 10 ++- .../google/cloud/operators/cloud_run.py | 8 +- .../google/cloud/operators/compute.py | 7 +- .../google/cloud/operators/datacatalog.py | 20 +++-- .../google/cloud/operators/dataform.py | 8 +- .../google/cloud/operators/datafusion.py | 2 +- .../google/cloud/operators/dataplex.py | 38 ++++---- .../google/cloud/operators/dataproc.py | 12 +-- .../cloud/operators/dataproc_metastore.py | 16 ++-- .../providers/google/cloud/operators/dlp.py | 36 ++++---- .../providers/google/cloud/operators/gcs.py | 5 +- .../cloud/operators/kubernetes_engine.py | 4 +- .../google/cloud/operators/managed_kafka.py | 10 ++- .../cloud/operators/natural_language.py | 8 +- .../google/cloud/operators/pubsub.py | 18 ++-- .../google/cloud/operators/speech_to_text.py | 8 +- .../google/cloud/operators/stackdriver.py | 8 +- .../providers/google/cloud/operators/tasks.py | 10 ++- .../google/cloud/operators/text_to_speech.py | 6 +- .../google/cloud/operators/translate.py | 8 +- .../cloud/operators/translate_speech.py | 6 +- .../cloud/operators/vertex_ai/auto_ml.py | 14 +-- .../vertex_ai/batch_prediction_job.py | 10 ++- .../cloud/operators/vertex_ai/custom_job.py | 14 +-- .../cloud/operators/vertex_ai/dataset.py | 10 ++- .../operators/vertex_ai/endpoint_service.py | 10 ++- .../vertex_ai/hyperparameter_tuning_job.py | 10 ++- .../operators/vertex_ai/model_service.py | 10 ++- .../cloud/operators/vertex_ai/pipeline_job.py | 10 ++- .../cloud/operators/video_intelligence.py | 8 +- .../google/cloud/operators/vision.py | 10 ++- .../google/cloud/operators/workflows.py | 12 +-- .../google/cloud/secrets/secret_manager.py | 3 +- .../google/cloud/sensors/bigquery_dts.py | 8 +- .../google/cloud/sensors/bigtable.py | 5 +- .../google/cloud/sensors/cloud_composer.py | 2 +- .../google/cloud/sensors/dataplex.py | 10 ++- .../google/cloud/sensors/dataproc.py | 5 +- .../cloud/sensors/dataproc_metastore.py | 3 +- .../providers/google/cloud/sensors/gcs.py | 6 +- .../providers/google/cloud/sensors/pubsub.py | 5 +- .../google/cloud/sensors/workflows.py | 8 +- .../google/cloud/transfers/bigquery_to_gcs.py | 8 +- .../google/cloud/transfers/gcs_to_bigquery.py | 22 ++--- .../google/cloud/triggers/bigquery_dts.py | 3 +- .../google/cloud/triggers/cloud_batch.py | 3 +- .../google/cloud/triggers/cloud_build.py | 3 +- .../google/cloud/triggers/cloud_composer.py | 2 +- .../cloud_storage_transfer_service.py | 5 +- .../google/cloud/triggers/dataflow.py | 5 +- .../google/cloud/triggers/dataplex.py | 3 +- .../google/cloud/triggers/dataproc.py | 5 +- .../cloud/triggers/kubernetes_engine.py | 2 +- .../providers/google/cloud/triggers/pubsub.py | 3 +- .../google/cloud/triggers/vertex_ai.py | 15 ++-- .../cloud/utils/credentials_provider.py | 7 +- .../cloud/utils/external_token_supplier.py | 1 - .../common/auth_backend/google_openid.py | 8 +- .../airflow/providers/google/common/consts.py | 3 +- .../google/common/hooks/base_google.py | 15 ++-- .../hooks/analytics_admin.py | 5 +- .../marketing_platform/hooks/search_ads.py | 2 +- .../operators/analytics_admin.py | 12 +-- providers/grpc/src/airflow/__init__.py | 17 ++++ .../grpc/src/airflow/providers/__init__.py | 17 ++++ .../src/airflow/providers/grpc/hooks/grpc.py | 2 +- providers/hashicorp/src/airflow/__init__.py | 17 ++++ .../src/airflow/providers/__init__.py | 17 ++++ providers/http/src/airflow/__init__.py | 17 ++++ .../http/src/airflow/providers/__init__.py | 17 ++++ providers/imap/src/airflow/__init__.py | 17 ++++ .../imap/src/airflow/providers/__init__.py | 17 ++++ providers/influxdb/src/airflow/__init__.py | 17 ++++ .../src/airflow/providers/__init__.py | 17 ++++ providers/jdbc/src/airflow/__init__.py | 17 ++++ .../jdbc/src/airflow/providers/__init__.py | 17 ++++ providers/jenkins/src/airflow/__init__.py | 17 ++++ .../jenkins/src/airflow/providers/__init__.py | 17 ++++ .../providers/jenkins/hooks/jenkins.py | 1 + .../jenkins/operators/jenkins_job_trigger.py | 4 +- .../microsoft/azure/src/airflow/__init__.py | 17 ++++ .../azure/src/airflow/providers/__init__.py | 17 ++++ .../airflow/providers/microsoft/__init__.py | 17 ++++ .../providers/microsoft/azure/fs/adls.py | 3 +- .../providers/microsoft/azure/hooks/adx.py | 5 +- .../providers/microsoft/azure/hooks/asb.py | 16 ++-- .../microsoft/azure/hooks/base_azure.py | 5 +- .../providers/microsoft/azure/hooks/batch.py | 3 +- .../azure/hooks/container_instance.py | 7 +- .../azure/hooks/container_registry.py | 5 +- .../microsoft/azure/hooks/container_volume.py | 5 +- .../providers/microsoft/azure/hooks/cosmos.py | 9 +- .../microsoft/azure/hooks/data_factory.py | 14 +-- .../microsoft/azure/hooks/data_lake.py | 17 ++-- .../microsoft/azure/hooks/fileshare.py | 3 +- .../microsoft/azure/hooks/msgraph.py | 4 +- .../microsoft/azure/hooks/synapse.py | 9 +- .../providers/microsoft/azure/hooks/wasb.py | 18 ++-- .../microsoft/azure/log/wasb_task_handler.py | 3 +- .../microsoft/azure/operators/adx.py | 3 +- .../microsoft/azure/operators/asb.py | 3 +- .../microsoft/azure/operators/batch.py | 3 +- .../azure/operators/container_instances.py | 14 +-- .../microsoft/azure/operators/synapse.py | 3 +- .../microsoft/azure/secrets/key_vault.py | 7 +- .../microsoft/azure/triggers/data_factory.py | 3 +- .../providers/microsoft/azure/utils.py | 3 +- .../microsoft/mssql/src/airflow/__init__.py | 17 ++++ .../mssql/src/airflow/providers/__init__.py | 17 ++++ .../airflow/providers/microsoft/__init__.py | 17 ++++ .../microsoft/psrp/src/airflow/__init__.py | 17 ++++ .../psrp/src/airflow/providers/__init__.py | 17 ++++ .../airflow/providers/microsoft/__init__.py | 17 ++++ .../microsoft/winrm/src/airflow/__init__.py | 17 ++++ .../winrm/src/airflow/providers/__init__.py | 17 ++++ .../airflow/providers/microsoft/__init__.py | 17 ++++ .../providers/microsoft/winrm/hooks/winrm.py | 5 +- providers/mongo/src/airflow/__init__.py | 17 ++++ .../mongo/src/airflow/providers/__init__.py | 17 ++++ providers/mysql/src/airflow/__init__.py | 17 ++++ .../mysql/src/airflow/providers/__init__.py | 17 ++++ providers/neo4j/src/airflow/__init__.py | 17 ++++ .../neo4j/src/airflow/providers/__init__.py | 17 ++++ .../airflow/providers/neo4j/hooks/neo4j.py | 3 +- providers/odbc/src/airflow/__init__.py | 17 ++++ .../odbc/src/airflow/providers/__init__.py | 17 ++++ providers/openai/src/airflow/__init__.py | 17 ++++ .../openai/src/airflow/providers/__init__.py | 17 ++++ providers/openfaas/src/airflow/__init__.py | 17 ++++ .../src/airflow/providers/__init__.py | 17 ++++ providers/openlineage/src/airflow/__init__.py | 17 ++++ .../src/airflow/providers/__init__.py | 17 ++++ .../providers/openlineage/extractors/base.py | 4 +- .../providers/openlineage/extractors/bash.py | 3 +- .../openlineage/extractors/manager.py | 6 +- .../openlineage/extractors/python.py | 3 +- .../providers/openlineage/plugins/adapter.py | 20 ++--- .../providers/openlineage/plugins/facets.py | 1 - .../providers/openlineage/plugins/listener.py | 2 +- .../providers/openlineage/sqlparser.py | 8 +- .../providers/openlineage/utils/sql.py | 3 +- .../providers/openlineage/utils/utils.py | 7 +- providers/opensearch/src/airflow/__init__.py | 17 ++++ .../src/airflow/providers/__init__.py | 17 ++++ providers/opsgenie/src/airflow/__init__.py | 17 ++++ .../src/airflow/providers/__init__.py | 17 ++++ providers/oracle/src/airflow/__init__.py | 17 ++++ .../oracle/src/airflow/providers/__init__.py | 17 ++++ providers/pagerduty/src/airflow/__init__.py | 17 ++++ .../src/airflow/providers/__init__.py | 17 ++++ providers/papermill/src/airflow/__init__.py | 17 ++++ .../src/airflow/providers/__init__.py | 17 ++++ .../providers/papermill/hooks/kernel.py | 6 +- .../papermill/operators/papermill.py | 2 +- providers/pgvector/src/airflow/__init__.py | 17 ++++ .../src/airflow/providers/__init__.py | 17 ++++ .../providers/pgvector/operators/pgvector.py | 3 +- providers/pinecone/src/airflow/__init__.py | 17 ++++ .../src/airflow/providers/__init__.py | 17 ++++ .../providers/pinecone/hooks/pinecone.py | 6 +- providers/postgres/src/airflow/__init__.py | 17 ++++ .../src/airflow/providers/__init__.py | 17 ++++ providers/presto/src/airflow/__init__.py | 17 ++++ .../presto/src/airflow/providers/__init__.py | 17 ++++ providers/qdrant/src/airflow/__init__.py | 17 ++++ .../qdrant/src/airflow/providers/__init__.py | 17 ++++ providers/redis/src/airflow/__init__.py | 17 ++++ .../redis/src/airflow/providers/__init__.py | 17 ++++ .../airflow/providers/redis/hooks/redis.py | 3 +- .../providers/redis/log/redis_task_handler.py | 3 +- providers/salesforce/src/airflow/__init__.py | 17 ++++ .../src/airflow/providers/__init__.py | 17 ++++ providers/samba/src/airflow/__init__.py | 17 ++++ .../samba/src/airflow/providers/__init__.py | 17 ++++ providers/segment/src/airflow/__init__.py | 17 ++++ .../segment/src/airflow/providers/__init__.py | 17 ++++ providers/sendgrid/src/airflow/__init__.py | 17 ++++ .../src/airflow/providers/__init__.py | 17 ++++ .../providers/sendgrid/utils/emailer.py | 5 +- providers/sftp/src/airflow/__init__.py | 17 ++++ .../sftp/src/airflow/providers/__init__.py | 17 ++++ providers/singularity/src/airflow/__init__.py | 17 ++++ .../src/airflow/providers/__init__.py | 17 ++++ providers/slack/src/airflow/__init__.py | 17 ++++ .../slack/src/airflow/providers/__init__.py | 17 ++++ providers/smtp/src/airflow/__init__.py | 17 ++++ .../smtp/src/airflow/providers/__init__.py | 17 ++++ providers/snowflake/src/airflow/__init__.py | 17 ++++ .../src/airflow/providers/__init__.py | 17 ++++ .../providers/snowflake/hooks/snowflake.py | 9 +- providers/sqlite/src/airflow/__init__.py | 17 ++++ .../sqlite/src/airflow/providers/__init__.py | 17 ++++ providers/ssh/src/airflow/__init__.py | 17 ++++ .../ssh/src/airflow/providers/__init__.py | 17 ++++ providers/standard/src/airflow/__init__.py | 17 ++++ .../src/airflow/providers/__init__.py | 17 ++++ providers/tableau/src/airflow/__init__.py | 17 ++++ .../tableau/src/airflow/providers/__init__.py | 17 ++++ providers/telegram/src/airflow/__init__.py | 17 ++++ .../src/airflow/providers/__init__.py | 17 ++++ .../providers/telegram/hooks/telegram.py | 2 +- providers/teradata/src/airflow/__init__.py | 17 ++++ .../src/airflow/providers/__init__.py | 17 ++++ providers/trino/src/airflow/__init__.py | 17 ++++ .../trino/src/airflow/providers/__init__.py | 17 ++++ .../airflow/providers/trino/hooks/trino.py | 5 +- providers/vertica/src/airflow/__init__.py | 17 ++++ .../vertica/src/airflow/providers/__init__.py | 17 ++++ providers/weaviate/src/airflow/__init__.py | 17 ++++ .../src/airflow/providers/__init__.py | 17 ++++ .../providers/weaviate/hooks/weaviate.py | 10 +-- .../providers/weaviate/operators/weaviate.py | 1 - providers/yandex/src/airflow/__init__.py | 17 ++++ .../yandex/src/airflow/providers/__init__.py | 17 ++++ .../providers/yandex/secrets/lockbox.py | 4 +- providers/ydb/src/airflow/__init__.py | 17 ++++ .../ydb/src/airflow/providers/__init__.py | 17 ++++ .../src/airflow/providers/ydb/hooks/ydb.py | 2 +- providers/zendesk/src/airflow/__init__.py | 17 ++++ .../zendesk/src/airflow/providers/__init__.py | 17 ++++ pyproject.toml | 17 ++-- ...eck_providers_subpackages_all_have_init.py | 56 +++++++----- .../ci/pre_commit/common_precommit_utils.py | 11 ++- scripts/ci/pre_commit/mypy.py | 6 +- scripts/in_container/run_mypy.sh | 1 + 444 files changed, 4545 insertions(+), 815 deletions(-) create mode 100644 providers/airbyte/src/airflow/__init__.py create mode 100644 providers/airbyte/src/airflow/providers/__init__.py create mode 100644 providers/alibaba/src/airflow/__init__.py create mode 100644 providers/alibaba/src/airflow/providers/__init__.py create mode 100644 providers/amazon/src/airflow/__init__.py create mode 100644 providers/amazon/src/airflow/providers/__init__.py create mode 100644 providers/apache/beam/src/airflow/__init__.py create mode 100644 providers/apache/beam/src/airflow/providers/__init__.py create mode 100644 providers/apache/beam/src/airflow/providers/apache/__init__.py create mode 100644 providers/apache/cassandra/src/airflow/__init__.py create mode 100644 providers/apache/cassandra/src/airflow/providers/__init__.py create mode 100644 providers/apache/cassandra/src/airflow/providers/apache/__init__.py create mode 100644 providers/apache/drill/src/airflow/__init__.py create mode 100644 providers/apache/drill/src/airflow/providers/__init__.py create mode 100644 providers/apache/drill/src/airflow/providers/apache/__init__.py create mode 100644 providers/apache/druid/src/airflow/__init__.py create mode 100644 providers/apache/druid/src/airflow/providers/__init__.py create mode 100644 providers/apache/druid/src/airflow/providers/apache/__init__.py create mode 100644 providers/apache/flink/src/airflow/__init__.py create mode 100644 providers/apache/flink/src/airflow/providers/__init__.py create mode 100644 providers/apache/flink/src/airflow/providers/apache/__init__.py create mode 100644 providers/apache/hdfs/src/airflow/__init__.py create mode 100644 providers/apache/hdfs/src/airflow/providers/__init__.py create mode 100644 providers/apache/hdfs/src/airflow/providers/apache/__init__.py create mode 100644 providers/apache/hive/src/airflow/__init__.py create mode 100644 providers/apache/hive/src/airflow/providers/__init__.py create mode 100644 providers/apache/hive/src/airflow/providers/apache/__init__.py create mode 100644 providers/apache/iceberg/src/airflow/__init__.py create mode 100644 providers/apache/iceberg/src/airflow/providers/__init__.py create mode 100644 providers/apache/iceberg/src/airflow/providers/apache/__init__.py create mode 100644 providers/apache/impala/src/airflow/__init__.py create mode 100644 providers/apache/impala/src/airflow/providers/__init__.py create mode 100644 providers/apache/impala/src/airflow/providers/apache/__init__.py create mode 100644 providers/apache/kafka/src/airflow/__init__.py create mode 100644 providers/apache/kafka/src/airflow/providers/__init__.py create mode 100644 providers/apache/kafka/src/airflow/providers/apache/__init__.py create mode 100644 providers/apache/kylin/src/airflow/__init__.py create mode 100644 providers/apache/kylin/src/airflow/providers/__init__.py create mode 100644 providers/apache/kylin/src/airflow/providers/apache/__init__.py create mode 100644 providers/apache/livy/src/airflow/__init__.py create mode 100644 providers/apache/livy/src/airflow/providers/__init__.py create mode 100644 providers/apache/livy/src/airflow/providers/apache/__init__.py create mode 100644 providers/apache/pig/src/airflow/__init__.py create mode 100644 providers/apache/pig/src/airflow/providers/__init__.py create mode 100644 providers/apache/pig/src/airflow/providers/apache/__init__.py create mode 100644 providers/apache/pinot/src/airflow/__init__.py create mode 100644 providers/apache/pinot/src/airflow/providers/__init__.py create mode 100644 providers/apache/pinot/src/airflow/providers/apache/__init__.py create mode 100644 providers/apache/spark/src/airflow/__init__.py create mode 100644 providers/apache/spark/src/airflow/providers/__init__.py create mode 100644 providers/apache/spark/src/airflow/providers/apache/__init__.py create mode 100644 providers/apprise/src/airflow/__init__.py create mode 100644 providers/apprise/src/airflow/providers/__init__.py create mode 100644 providers/arangodb/src/airflow/__init__.py create mode 100644 providers/arangodb/src/airflow/providers/__init__.py create mode 100644 providers/asana/src/airflow/__init__.py create mode 100644 providers/asana/src/airflow/providers/__init__.py create mode 100644 providers/atlassian/jira/src/airflow/__init__.py create mode 100644 providers/atlassian/jira/src/airflow/providers/__init__.py create mode 100644 providers/atlassian/jira/src/airflow/providers/atlassian/__init__.py create mode 100644 providers/celery/src/airflow/__init__.py create mode 100644 providers/celery/src/airflow/providers/__init__.py create mode 100644 providers/cloudant/src/airflow/__init__.py create mode 100644 providers/cloudant/src/airflow/providers/__init__.py create mode 100644 providers/cncf/kubernetes/src/airflow/__init__.py create mode 100644 providers/cncf/kubernetes/src/airflow/providers/__init__.py create mode 100644 providers/cncf/kubernetes/src/airflow/providers/cncf/__init__.py create mode 100644 providers/cohere/src/airflow/__init__.py create mode 100644 providers/cohere/src/airflow/providers/__init__.py create mode 100644 providers/common/compat/src/airflow/__init__.py create mode 100644 providers/common/compat/src/airflow/providers/__init__.py create mode 100644 providers/common/compat/src/airflow/providers/common/__init__.py create mode 100644 providers/common/io/src/airflow/__init__.py create mode 100644 providers/common/io/src/airflow/providers/__init__.py create mode 100644 providers/common/io/src/airflow/providers/common/__init__.py create mode 100644 providers/common/sql/src/airflow/__init__.py create mode 100644 providers/common/sql/src/airflow/providers/__init__.py create mode 100644 providers/common/sql/src/airflow/providers/common/__init__.py create mode 100644 providers/databricks/src/airflow/__init__.py create mode 100644 providers/databricks/src/airflow/providers/__init__.py create mode 100644 providers/datadog/src/airflow/__init__.py create mode 100644 providers/datadog/src/airflow/providers/__init__.py create mode 100644 providers/dbt/cloud/src/airflow/__init__.py create mode 100644 providers/dbt/cloud/src/airflow/providers/__init__.py create mode 100644 providers/dbt/cloud/src/airflow/providers/dbt/__init__.py create mode 100644 providers/dingding/src/airflow/__init__.py create mode 100644 providers/dingding/src/airflow/providers/__init__.py create mode 100644 providers/discord/src/airflow/__init__.py create mode 100644 providers/discord/src/airflow/providers/__init__.py create mode 100644 providers/docker/src/airflow/__init__.py create mode 100644 providers/docker/src/airflow/providers/__init__.py create mode 100644 providers/edge/src/airflow/__init__.py create mode 100644 providers/edge/src/airflow/providers/__init__.py create mode 100644 providers/elasticsearch/src/airflow/__init__.py create mode 100644 providers/elasticsearch/src/airflow/providers/__init__.py create mode 100644 providers/exasol/src/airflow/__init__.py create mode 100644 providers/exasol/src/airflow/providers/__init__.py create mode 100644 providers/fab/src/airflow/__init__.py delete mode 100644 providers/fab/src/airflow/providers/3rd-party-licenses/LICENSES-ui.txt create mode 100644 providers/fab/src/airflow/providers/__init__.py create mode 100644 providers/facebook/src/airflow/__init__.py create mode 100644 providers/facebook/src/airflow/providers/__init__.py create mode 100644 providers/ftp/src/airflow/__init__.py create mode 100644 providers/ftp/src/airflow/providers/__init__.py create mode 100644 providers/github/src/airflow/__init__.py create mode 100644 providers/github/src/airflow/providers/__init__.py create mode 100644 providers/google/src/airflow/__init__.py create mode 100644 providers/google/src/airflow/providers/__init__.py create mode 100644 providers/grpc/src/airflow/__init__.py create mode 100644 providers/grpc/src/airflow/providers/__init__.py create mode 100644 providers/hashicorp/src/airflow/__init__.py create mode 100644 providers/hashicorp/src/airflow/providers/__init__.py create mode 100644 providers/http/src/airflow/__init__.py create mode 100644 providers/http/src/airflow/providers/__init__.py create mode 100644 providers/imap/src/airflow/__init__.py create mode 100644 providers/imap/src/airflow/providers/__init__.py create mode 100644 providers/influxdb/src/airflow/__init__.py create mode 100644 providers/influxdb/src/airflow/providers/__init__.py create mode 100644 providers/jdbc/src/airflow/__init__.py create mode 100644 providers/jdbc/src/airflow/providers/__init__.py create mode 100644 providers/jenkins/src/airflow/__init__.py create mode 100644 providers/jenkins/src/airflow/providers/__init__.py create mode 100644 providers/microsoft/azure/src/airflow/__init__.py create mode 100644 providers/microsoft/azure/src/airflow/providers/__init__.py create mode 100644 providers/microsoft/azure/src/airflow/providers/microsoft/__init__.py create mode 100644 providers/microsoft/mssql/src/airflow/__init__.py create mode 100644 providers/microsoft/mssql/src/airflow/providers/__init__.py create mode 100644 providers/microsoft/mssql/src/airflow/providers/microsoft/__init__.py create mode 100644 providers/microsoft/psrp/src/airflow/__init__.py create mode 100644 providers/microsoft/psrp/src/airflow/providers/__init__.py create mode 100644 providers/microsoft/psrp/src/airflow/providers/microsoft/__init__.py create mode 100644 providers/microsoft/winrm/src/airflow/__init__.py create mode 100644 providers/microsoft/winrm/src/airflow/providers/__init__.py create mode 100644 providers/microsoft/winrm/src/airflow/providers/microsoft/__init__.py create mode 100644 providers/mongo/src/airflow/__init__.py create mode 100644 providers/mongo/src/airflow/providers/__init__.py create mode 100644 providers/mysql/src/airflow/__init__.py create mode 100644 providers/mysql/src/airflow/providers/__init__.py create mode 100644 providers/neo4j/src/airflow/__init__.py create mode 100644 providers/neo4j/src/airflow/providers/__init__.py create mode 100644 providers/odbc/src/airflow/__init__.py create mode 100644 providers/odbc/src/airflow/providers/__init__.py create mode 100644 providers/openai/src/airflow/__init__.py create mode 100644 providers/openai/src/airflow/providers/__init__.py create mode 100644 providers/openfaas/src/airflow/__init__.py create mode 100644 providers/openfaas/src/airflow/providers/__init__.py create mode 100644 providers/openlineage/src/airflow/__init__.py create mode 100644 providers/openlineage/src/airflow/providers/__init__.py create mode 100644 providers/opensearch/src/airflow/__init__.py create mode 100644 providers/opensearch/src/airflow/providers/__init__.py create mode 100644 providers/opsgenie/src/airflow/__init__.py create mode 100644 providers/opsgenie/src/airflow/providers/__init__.py create mode 100644 providers/oracle/src/airflow/__init__.py create mode 100644 providers/oracle/src/airflow/providers/__init__.py create mode 100644 providers/pagerduty/src/airflow/__init__.py create mode 100644 providers/pagerduty/src/airflow/providers/__init__.py create mode 100644 providers/papermill/src/airflow/__init__.py create mode 100644 providers/papermill/src/airflow/providers/__init__.py create mode 100644 providers/pgvector/src/airflow/__init__.py create mode 100644 providers/pgvector/src/airflow/providers/__init__.py create mode 100644 providers/pinecone/src/airflow/__init__.py create mode 100644 providers/pinecone/src/airflow/providers/__init__.py create mode 100644 providers/postgres/src/airflow/__init__.py create mode 100644 providers/postgres/src/airflow/providers/__init__.py create mode 100644 providers/presto/src/airflow/__init__.py create mode 100644 providers/presto/src/airflow/providers/__init__.py create mode 100644 providers/qdrant/src/airflow/__init__.py create mode 100644 providers/qdrant/src/airflow/providers/__init__.py create mode 100644 providers/redis/src/airflow/__init__.py create mode 100644 providers/redis/src/airflow/providers/__init__.py create mode 100644 providers/salesforce/src/airflow/__init__.py create mode 100644 providers/salesforce/src/airflow/providers/__init__.py create mode 100644 providers/samba/src/airflow/__init__.py create mode 100644 providers/samba/src/airflow/providers/__init__.py create mode 100644 providers/segment/src/airflow/__init__.py create mode 100644 providers/segment/src/airflow/providers/__init__.py create mode 100644 providers/sendgrid/src/airflow/__init__.py create mode 100644 providers/sendgrid/src/airflow/providers/__init__.py create mode 100644 providers/sftp/src/airflow/__init__.py create mode 100644 providers/sftp/src/airflow/providers/__init__.py create mode 100644 providers/singularity/src/airflow/__init__.py create mode 100644 providers/singularity/src/airflow/providers/__init__.py create mode 100644 providers/slack/src/airflow/__init__.py create mode 100644 providers/slack/src/airflow/providers/__init__.py create mode 100644 providers/smtp/src/airflow/__init__.py create mode 100644 providers/smtp/src/airflow/providers/__init__.py create mode 100644 providers/snowflake/src/airflow/__init__.py create mode 100644 providers/snowflake/src/airflow/providers/__init__.py create mode 100644 providers/sqlite/src/airflow/__init__.py create mode 100644 providers/sqlite/src/airflow/providers/__init__.py create mode 100644 providers/ssh/src/airflow/__init__.py create mode 100644 providers/ssh/src/airflow/providers/__init__.py create mode 100644 providers/standard/src/airflow/__init__.py create mode 100644 providers/standard/src/airflow/providers/__init__.py create mode 100644 providers/tableau/src/airflow/__init__.py create mode 100644 providers/tableau/src/airflow/providers/__init__.py create mode 100644 providers/telegram/src/airflow/__init__.py create mode 100644 providers/telegram/src/airflow/providers/__init__.py create mode 100644 providers/teradata/src/airflow/__init__.py create mode 100644 providers/teradata/src/airflow/providers/__init__.py create mode 100644 providers/trino/src/airflow/__init__.py create mode 100644 providers/trino/src/airflow/providers/__init__.py create mode 100644 providers/vertica/src/airflow/__init__.py create mode 100644 providers/vertica/src/airflow/providers/__init__.py create mode 100644 providers/weaviate/src/airflow/__init__.py create mode 100644 providers/weaviate/src/airflow/providers/__init__.py create mode 100644 providers/yandex/src/airflow/__init__.py create mode 100644 providers/yandex/src/airflow/providers/__init__.py create mode 100644 providers/ydb/src/airflow/__init__.py create mode 100644 providers/ydb/src/airflow/providers/__init__.py create mode 100644 providers/zendesk/src/airflow/__init__.py create mode 100644 providers/zendesk/src/airflow/providers/__init__.py diff --git a/docs/conf.py b/docs/conf.py index 50aa390cbd6e8..f20fe940e19e2 100644 --- a/docs/conf.py +++ b/docs/conf.py @@ -223,6 +223,14 @@ empty_subpackages = ["apache", "atlassian", "common", "cncf", "dbt", "microsoft"] exclude_patterns = [ "operators/_partials", + "_api/airflow/index.rst", + "_api/airflow/providers/index.rst", + "_api/airflow/providers/apache/index.rst", + "_api/airflow/providers/atlassian/index.rst", + "_api/airflow/providers/cncf/index.rst", + "_api/airflow/providers/common/index.rst", + "_api/airflow/providers/dbt/index.rst", + "_api/airflow/providers/microsoft/index.rst", *[f"_api/tests/system/{subpackage}/index.rst" for subpackage in empty_subpackages], ] else: diff --git a/providers/airbyte/src/airflow/__init__.py b/providers/airbyte/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/airbyte/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/airbyte/src/airflow/providers/__init__.py b/providers/airbyte/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/airbyte/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/alibaba/src/airflow/__init__.py b/providers/alibaba/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/alibaba/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/alibaba/src/airflow/providers/__init__.py b/providers/alibaba/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/alibaba/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/amazon/src/airflow/__init__.py b/providers/amazon/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/amazon/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/amazon/src/airflow/providers/__init__.py b/providers/amazon/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/amazon/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/beam/src/airflow/__init__.py b/providers/apache/beam/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/beam/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/beam/src/airflow/providers/__init__.py b/providers/apache/beam/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/beam/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/beam/src/airflow/providers/apache/__init__.py b/providers/apache/beam/src/airflow/providers/apache/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/beam/src/airflow/providers/apache/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/cassandra/src/airflow/__init__.py b/providers/apache/cassandra/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/cassandra/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/cassandra/src/airflow/providers/__init__.py b/providers/apache/cassandra/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/cassandra/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/cassandra/src/airflow/providers/apache/__init__.py b/providers/apache/cassandra/src/airflow/providers/apache/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/cassandra/src/airflow/providers/apache/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/cassandra/src/airflow/providers/apache/cassandra/hooks/cassandra.py b/providers/apache/cassandra/src/airflow/providers/apache/cassandra/hooks/cassandra.py index 0727404e9884b..0d2279b754ce3 100644 --- a/providers/apache/cassandra/src/airflow/providers/apache/cassandra/hooks/cassandra.py +++ b/providers/apache/cassandra/src/airflow/providers/apache/cassandra/hooks/cassandra.py @@ -22,8 +22,6 @@ import re from typing import Any, Union -from airflow.hooks.base import BaseHook -from airflow.utils.log.logging_mixin import LoggingMixin from cassandra.auth import PlainTextAuthProvider from cassandra.cluster import Cluster, Session from cassandra.policies import ( @@ -33,6 +31,9 @@ WhiteListRoundRobinPolicy, ) +from airflow.hooks.base import BaseHook +from airflow.utils.log.logging_mixin import LoggingMixin + Policy = Union[DCAwareRoundRobinPolicy, RoundRobinPolicy, TokenAwarePolicy, WhiteListRoundRobinPolicy] diff --git a/providers/apache/drill/src/airflow/__init__.py b/providers/apache/drill/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/drill/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/drill/src/airflow/providers/__init__.py b/providers/apache/drill/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/drill/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/drill/src/airflow/providers/apache/__init__.py b/providers/apache/drill/src/airflow/providers/apache/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/drill/src/airflow/providers/apache/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/druid/src/airflow/__init__.py b/providers/apache/druid/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/druid/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/druid/src/airflow/providers/__init__.py b/providers/apache/druid/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/druid/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/druid/src/airflow/providers/apache/__init__.py b/providers/apache/druid/src/airflow/providers/apache/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/druid/src/airflow/providers/apache/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/flink/src/airflow/__init__.py b/providers/apache/flink/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/flink/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/flink/src/airflow/providers/__init__.py b/providers/apache/flink/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/flink/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/flink/src/airflow/providers/apache/__init__.py b/providers/apache/flink/src/airflow/providers/apache/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/flink/src/airflow/providers/apache/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/hdfs/src/airflow/__init__.py b/providers/apache/hdfs/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/hdfs/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/hdfs/src/airflow/providers/__init__.py b/providers/apache/hdfs/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/hdfs/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/hdfs/src/airflow/providers/apache/__init__.py b/providers/apache/hdfs/src/airflow/providers/apache/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/hdfs/src/airflow/providers/apache/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/hdfs/src/airflow/providers/apache/hdfs/hooks/webhdfs.py b/providers/apache/hdfs/src/airflow/providers/apache/hdfs/hooks/webhdfs.py index f1ff1e1655f44..47ab97db4e8df 100644 --- a/providers/apache/hdfs/src/airflow/providers/apache/hdfs/hooks/webhdfs.py +++ b/providers/apache/hdfs/src/airflow/providers/apache/hdfs/hooks/webhdfs.py @@ -24,11 +24,11 @@ from typing import Any import requests +from hdfs import HdfsError, InsecureClient from airflow.configuration import conf from airflow.exceptions import AirflowException from airflow.hooks.base import BaseHook -from hdfs import HdfsError, InsecureClient log = logging.getLogger(__name__) diff --git a/providers/apache/hdfs/src/airflow/providers/apache/hdfs/sensors/web_hdfs.py b/providers/apache/hdfs/src/airflow/providers/apache/hdfs/sensors/web_hdfs.py index 2af323bbf5f62..8b951bf3ec835 100644 --- a/providers/apache/hdfs/src/airflow/providers/apache/hdfs/sensors/web_hdfs.py +++ b/providers/apache/hdfs/src/airflow/providers/apache/hdfs/sensors/web_hdfs.py @@ -23,10 +23,11 @@ from airflow.sensors.base import BaseSensorOperator if TYPE_CHECKING: - from airflow.utils.context import Context from hdfs import InsecureClient from hdfs.ext.kerberos import KerberosClient + from airflow.utils.context import Context + class WebHdfsSensor(BaseSensorOperator): """Waits for a file or folder to land in HDFS.""" diff --git a/providers/apache/hive/src/airflow/__init__.py b/providers/apache/hive/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/hive/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/hive/src/airflow/providers/__init__.py b/providers/apache/hive/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/hive/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/hive/src/airflow/providers/apache/__init__.py b/providers/apache/hive/src/airflow/providers/apache/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/hive/src/airflow/providers/apache/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/iceberg/src/airflow/__init__.py b/providers/apache/iceberg/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/iceberg/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/iceberg/src/airflow/providers/__init__.py b/providers/apache/iceberg/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/iceberg/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/iceberg/src/airflow/providers/apache/__init__.py b/providers/apache/iceberg/src/airflow/providers/apache/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/iceberg/src/airflow/providers/apache/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/impala/src/airflow/__init__.py b/providers/apache/impala/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/impala/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/impala/src/airflow/providers/__init__.py b/providers/apache/impala/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/impala/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/impala/src/airflow/providers/apache/__init__.py b/providers/apache/impala/src/airflow/providers/apache/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/impala/src/airflow/providers/apache/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/impala/src/airflow/providers/apache/impala/hooks/impala.py b/providers/apache/impala/src/airflow/providers/apache/impala/hooks/impala.py index e95eeac908c7c..aaa510945553b 100644 --- a/providers/apache/impala/src/airflow/providers/apache/impala/hooks/impala.py +++ b/providers/apache/impala/src/airflow/providers/apache/impala/hooks/impala.py @@ -18,9 +18,10 @@ from typing import TYPE_CHECKING -from airflow.providers.common.sql.hooks.sql import DbApiHook from impala.dbapi import connect +from airflow.providers.common.sql.hooks.sql import DbApiHook + if TYPE_CHECKING: from impala.interface import Connection diff --git a/providers/apache/kafka/src/airflow/__init__.py b/providers/apache/kafka/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/kafka/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/kafka/src/airflow/providers/__init__.py b/providers/apache/kafka/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/kafka/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/kafka/src/airflow/providers/apache/__init__.py b/providers/apache/kafka/src/airflow/providers/apache/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/kafka/src/airflow/providers/apache/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/kylin/src/airflow/__init__.py b/providers/apache/kylin/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/kylin/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/kylin/src/airflow/providers/__init__.py b/providers/apache/kylin/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/kylin/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/kylin/src/airflow/providers/apache/__init__.py b/providers/apache/kylin/src/airflow/providers/apache/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/kylin/src/airflow/providers/apache/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/livy/src/airflow/__init__.py b/providers/apache/livy/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/livy/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/livy/src/airflow/providers/__init__.py b/providers/apache/livy/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/livy/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/livy/src/airflow/providers/apache/__init__.py b/providers/apache/livy/src/airflow/providers/apache/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/livy/src/airflow/providers/apache/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/pig/src/airflow/__init__.py b/providers/apache/pig/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/pig/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/pig/src/airflow/providers/__init__.py b/providers/apache/pig/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/pig/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/pig/src/airflow/providers/apache/__init__.py b/providers/apache/pig/src/airflow/providers/apache/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/pig/src/airflow/providers/apache/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/pinot/src/airflow/__init__.py b/providers/apache/pinot/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/pinot/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/pinot/src/airflow/providers/__init__.py b/providers/apache/pinot/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/pinot/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/pinot/src/airflow/providers/apache/__init__.py b/providers/apache/pinot/src/airflow/providers/apache/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/pinot/src/airflow/providers/apache/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/spark/src/airflow/__init__.py b/providers/apache/spark/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/spark/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/spark/src/airflow/providers/__init__.py b/providers/apache/spark/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/spark/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apache/spark/src/airflow/providers/apache/__init__.py b/providers/apache/spark/src/airflow/providers/apache/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apache/spark/src/airflow/providers/apache/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apprise/src/airflow/__init__.py b/providers/apprise/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apprise/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apprise/src/airflow/providers/__init__.py b/providers/apprise/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/apprise/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/apprise/src/airflow/providers/apprise/hooks/apprise.py b/providers/apprise/src/airflow/providers/apprise/hooks/apprise.py index 831fde2b46ddd..4fe373281f6c7 100644 --- a/providers/apprise/src/airflow/providers/apprise/hooks/apprise.py +++ b/providers/apprise/src/airflow/providers/apprise/hooks/apprise.py @@ -22,9 +22,10 @@ from typing import TYPE_CHECKING, Any import apprise -from airflow.hooks.base import BaseHook from apprise import AppriseConfig, NotifyFormat, NotifyType +from airflow.hooks.base import BaseHook + if TYPE_CHECKING: from apprise import AppriseAttachment diff --git a/providers/apprise/src/airflow/providers/apprise/notifications/apprise.py b/providers/apprise/src/airflow/providers/apprise/notifications/apprise.py index 36abafb6fab4e..a1fe7e3c4be17 100644 --- a/providers/apprise/src/airflow/providers/apprise/notifications/apprise.py +++ b/providers/apprise/src/airflow/providers/apprise/notifications/apprise.py @@ -20,9 +20,10 @@ from collections.abc import Iterable from functools import cached_property +from apprise import AppriseConfig, NotifyFormat, NotifyType + from airflow.notifications.basenotifier import BaseNotifier from airflow.providers.apprise.hooks.apprise import AppriseHook -from apprise import AppriseConfig, NotifyFormat, NotifyType class AppriseNotifier(BaseNotifier): diff --git a/providers/arangodb/src/airflow/__init__.py b/providers/arangodb/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/arangodb/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/arangodb/src/airflow/providers/__init__.py b/providers/arangodb/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/arangodb/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/asana/src/airflow/__init__.py b/providers/asana/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/asana/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/asana/src/airflow/providers/__init__.py b/providers/asana/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/asana/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/asana/src/airflow/providers/asana/hooks/asana.py b/providers/asana/src/airflow/providers/asana/hooks/asana.py index 8416d107fde3f..dc78ab0ee4e20 100644 --- a/providers/asana/src/airflow/providers/asana/hooks/asana.py +++ b/providers/asana/src/airflow/providers/asana/hooks/asana.py @@ -22,13 +22,14 @@ from functools import cached_property from typing import Any -from airflow.hooks.base import BaseHook from asana.api.projects_api import ProjectsApi from asana.api.tasks_api import TasksApi from asana.api_client import ApiClient from asana.configuration import Configuration from asana.rest import ApiException +from airflow.hooks.base import BaseHook + class AsanaHook(BaseHook): """Wrapper around Asana Python client library.""" diff --git a/providers/atlassian/jira/src/airflow/__init__.py b/providers/atlassian/jira/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/atlassian/jira/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/atlassian/jira/src/airflow/providers/__init__.py b/providers/atlassian/jira/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/atlassian/jira/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/atlassian/jira/src/airflow/providers/atlassian/__init__.py b/providers/atlassian/jira/src/airflow/providers/atlassian/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/atlassian/jira/src/airflow/providers/atlassian/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/celery/src/airflow/__init__.py b/providers/celery/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/celery/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/celery/src/airflow/providers/__init__.py b/providers/celery/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/celery/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/celery/src/airflow/providers/celery/cli/celery_command.py b/providers/celery/src/airflow/providers/celery/cli/celery_command.py index aa0a0ec2ebe57..b935523c89da4 100644 --- a/providers/celery/src/airflow/providers/celery/cli/celery_command.py +++ b/providers/celery/src/airflow/providers/celery/cli/celery_command.py @@ -26,6 +26,9 @@ import psutil import sqlalchemy.exc +from celery import maybe_patch_concurrency # type: ignore[attr-defined] +from celery.app.defaults import DEFAULT_TASK_LOG_FMT +from celery.signals import after_setup_logger from lockfile.pidlockfile import read_pid_from_pidfile, remove_existing_pidfile from airflow import settings @@ -34,9 +37,6 @@ from airflow.utils import cli as cli_utils from airflow.utils.cli import setup_locations from airflow.utils.serve_logs import serve_logs -from celery import maybe_patch_concurrency # type: ignore[attr-defined] -from celery.app.defaults import DEFAULT_TASK_LOG_FMT -from celery.signals import after_setup_logger WORKER_PROCESS_NAME = "worker" diff --git a/providers/celery/src/airflow/providers/celery/executors/celery_executor.py b/providers/celery/src/airflow/providers/celery/executors/celery_executor.py index 0d88eb61b8a1d..f442658a069c6 100644 --- a/providers/celery/src/airflow/providers/celery/executors/celery_executor.py +++ b/providers/celery/src/airflow/providers/celery/executors/celery_executor.py @@ -35,6 +35,7 @@ from multiprocessing import cpu_count from typing import TYPE_CHECKING, Any +from celery import states as celery_states from deprecated import deprecated from airflow.cli.cli_config import ( @@ -56,7 +57,6 @@ from airflow.providers.celery.version_compat import AIRFLOW_V_3_0_PLUS from airflow.stats import Stats from airflow.utils.state import TaskInstanceState -from celery import states as celery_states log = logging.getLogger(__name__) diff --git a/providers/celery/src/airflow/providers/celery/executors/celery_executor_utils.py b/providers/celery/src/airflow/providers/celery/executors/celery_executor_utils.py index ee73c2cab95bf..93f801356391f 100644 --- a/providers/celery/src/airflow/providers/celery/executors/celery_executor_utils.py +++ b/providers/celery/src/airflow/providers/celery/executors/celery_executor_utils.py @@ -33,6 +33,10 @@ from concurrent.futures import ProcessPoolExecutor from typing import TYPE_CHECKING, Any, Optional, Union +from celery import Celery, Task, states as celery_states +from celery.backends.base import BaseKeyValueStoreBackend +from celery.backends.database import DatabaseBackend, Task as TaskDb, retry, session_cleanup +from celery.signals import import_modules as celery_import_modules from setproctitle import setproctitle from sqlalchemy import select @@ -46,10 +50,6 @@ from airflow.utils.net import get_hostname from airflow.utils.providers_configuration_loader import providers_configuration_loaded from airflow.utils.timeout import timeout -from celery import Celery, Task, states as celery_states -from celery.backends.base import BaseKeyValueStoreBackend -from celery.backends.database import DatabaseBackend, Task as TaskDb, retry, session_cleanup -from celery.signals import import_modules as celery_import_modules try: from airflow.sdk.definitions._internal.dag_parsing_context import _airflow_parsing_context_manager @@ -59,11 +59,12 @@ log = logging.getLogger(__name__) if TYPE_CHECKING: + from celery.result import AsyncResult + from airflow.executors import workloads from airflow.executors.base_executor import CommandType, EventBufferValueType from airflow.models.taskinstance import TaskInstanceKey from airflow.typing_compat import TypeAlias - from celery.result import AsyncResult # We can't use `if AIRFLOW_V_3_0_PLUS` conditions in type checks, so unfortunately we just have to define # the type as the union of both kinds diff --git a/providers/celery/src/airflow/providers/celery/sensors/celery_queue.py b/providers/celery/src/airflow/providers/celery/sensors/celery_queue.py index 4efad5b6745ea..c8c3b3d131162 100644 --- a/providers/celery/src/airflow/providers/celery/sensors/celery_queue.py +++ b/providers/celery/src/airflow/providers/celery/sensors/celery_queue.py @@ -19,9 +19,10 @@ from typing import TYPE_CHECKING -from airflow.sensors.base import BaseSensorOperator from celery.app import control +from airflow.sensors.base import BaseSensorOperator + if TYPE_CHECKING: try: from airflow.sdk.definitions.context import Context diff --git a/providers/cloudant/src/airflow/__init__.py b/providers/cloudant/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/cloudant/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/cloudant/src/airflow/providers/__init__.py b/providers/cloudant/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/cloudant/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/cncf/kubernetes/src/airflow/__init__.py b/providers/cncf/kubernetes/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/cncf/kubernetes/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/cncf/kubernetes/src/airflow/providers/__init__.py b/providers/cncf/kubernetes/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/cncf/kubernetes/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/cncf/kubernetes/src/airflow/providers/cncf/__init__.py b/providers/cncf/kubernetes/src/airflow/providers/cncf/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/cncf/kubernetes/src/airflow/providers/cncf/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/backcompat/backwards_compat_converters.py b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/backcompat/backwards_compat_converters.py index c09291a70d505..97483b0865315 100644 --- a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/backcompat/backwards_compat_converters.py +++ b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/backcompat/backwards_compat_converters.py @@ -18,9 +18,10 @@ from __future__ import annotations -from airflow.exceptions import AirflowException from kubernetes.client import ApiClient, models as k8s +from airflow.exceptions import AirflowException + def _convert_kube_model_object(obj, new_class): convert_op = getattr(obj, "to_k8s_client_obj", None) diff --git a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/callbacks.py b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/callbacks.py index 723de2f94f990..d87e8065dbd1a 100644 --- a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/callbacks.py +++ b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/callbacks.py @@ -19,9 +19,8 @@ from enum import Enum from typing import TYPE_CHECKING, Union -import kubernetes_asyncio.client as async_k8s - import kubernetes.client as k8s +import kubernetes_asyncio.client as async_k8s if TYPE_CHECKING: from airflow.providers.cncf.kubernetes.operators.pod import KubernetesPodOperator diff --git a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/cli/kubernetes_command.py b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/cli/kubernetes_command.py index d138fd9c77c00..ee629cc16e9a5 100644 --- a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/cli/kubernetes_command.py +++ b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/cli/kubernetes_command.py @@ -22,6 +22,10 @@ import sys from datetime import datetime, timedelta +from kubernetes import client +from kubernetes.client.api_client import ApiClient +from kubernetes.client.rest import ApiException + from airflow.models import DagRun, TaskInstance from airflow.providers.cncf.kubernetes import pod_generator from airflow.providers.cncf.kubernetes.executors.kubernetes_executor import KubeConfig @@ -32,9 +36,6 @@ from airflow.utils import cli as cli_utils, yaml from airflow.utils.cli import get_dag from airflow.utils.providers_configuration_loader import providers_configuration_loaded -from kubernetes import client -from kubernetes.client.api_client import ApiClient -from kubernetes.client.rest import ApiException @cli_utils.action_cli diff --git a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/decorators/kubernetes.py b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/decorators/kubernetes.py index d2b3385fe536a..7e64f01559395 100644 --- a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/decorators/kubernetes.py +++ b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/decorators/kubernetes.py @@ -25,13 +25,13 @@ from typing import TYPE_CHECKING, Callable import dill +from kubernetes.client import models as k8s from airflow.decorators.base import DecoratedOperator, TaskDecorator, task_decorator_factory from airflow.providers.cncf.kubernetes.operators.pod import KubernetesPodOperator from airflow.providers.cncf.kubernetes.python_kubernetes_script import ( write_python_script, ) -from kubernetes.client import models as k8s if TYPE_CHECKING: from airflow.utils.context import Context diff --git a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/executors/kubernetes_executor.py b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/executors/kubernetes_executor.py index 21b18ef45711a..7a36666e9fab2 100644 --- a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/executors/kubernetes_executor.py +++ b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/executors/kubernetes_executor.py @@ -37,11 +37,11 @@ from typing import TYPE_CHECKING, Any from deprecated import deprecated +from kubernetes.dynamic import DynamicClient from sqlalchemy import select from airflow.providers.cncf.kubernetes.pod_generator import PodGenerator from airflow.providers.cncf.kubernetes.version_compat import AIRFLOW_V_3_0_PLUS -from kubernetes.dynamic import DynamicClient try: from airflow.cli.cli_config import ARG_LOGICAL_DATE @@ -78,6 +78,8 @@ if TYPE_CHECKING: import argparse + from kubernetes import client + from kubernetes.client import models as k8s from sqlalchemy.orm import Session from airflow.executors import workloads @@ -91,8 +93,6 @@ from airflow.providers.cncf.kubernetes.executors.kubernetes_executor_utils import ( AirflowKubernetesScheduler, ) - from kubernetes import client - from kubernetes.client import models as k8s # CLI Args ARG_NAMESPACE = Arg( diff --git a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/executors/kubernetes_executor_utils.py b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/executors/kubernetes_executor_utils.py index 6a98bdfda3ab3..a59f1e47d1ba7 100644 --- a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/executors/kubernetes_executor_utils.py +++ b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/executors/kubernetes_executor_utils.py @@ -23,6 +23,8 @@ from queue import Empty, Queue from typing import TYPE_CHECKING, Any +from kubernetes import client, watch +from kubernetes.client.rest import ApiException from urllib3.exceptions import ReadTimeoutError from airflow.exceptions import AirflowException @@ -43,16 +45,15 @@ from airflow.utils.log.logging_mixin import LoggingMixin from airflow.utils.singleton import Singleton from airflow.utils.state import TaskInstanceState -from kubernetes import client, watch -from kubernetes.client.rest import ApiException if TYPE_CHECKING: + from kubernetes.client import Configuration, models as k8s + from airflow.providers.cncf.kubernetes.executors.kubernetes_executor_types import ( KubernetesJobType, KubernetesResultsType, KubernetesWatchType, ) - from kubernetes.client import Configuration, models as k8s class ResourceVersion(metaclass=Singleton): diff --git a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/hooks/kubernetes.py b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/hooks/kubernetes.py index 4b858e796b316..4adef6ba20e69 100644 --- a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/hooks/kubernetes.py +++ b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/hooks/kubernetes.py @@ -29,6 +29,9 @@ import requests import tenacity from asgiref.sync import sync_to_async +from kubernetes import client, config, utils, watch +from kubernetes.client.models import V1Deployment +from kubernetes.config import ConfigException from kubernetes_asyncio import client as async_client, config as async_config from urllib3.exceptions import HTTPError @@ -43,9 +46,6 @@ container_is_running, ) from airflow.utils import yaml -from kubernetes import client, config, utils, watch -from kubernetes.client.models import V1Deployment -from kubernetes.config import ConfigException if TYPE_CHECKING: from kubernetes.client import V1JobList diff --git a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/kubernetes_helper_functions.py b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/kubernetes_helper_functions.py index 2732907886970..a353b7a5c8026 100644 --- a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/kubernetes_helper_functions.py +++ b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/kubernetes_helper_functions.py @@ -23,11 +23,11 @@ from typing import TYPE_CHECKING import pendulum +from kubernetes.client.rest import ApiException from slugify import slugify from airflow.configuration import conf from airflow.providers.cncf.kubernetes.backcompat import get_logical_date_key -from kubernetes.client.rest import ApiException if TYPE_CHECKING: from airflow.models.taskinstancekey import TaskInstanceKey diff --git a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/operators/custom_object_launcher.py b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/operators/custom_object_launcher.py index 33c204b8d3f19..c6d1d14cc624e 100644 --- a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/operators/custom_object_launcher.py +++ b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/operators/custom_object_launcher.py @@ -24,6 +24,8 @@ from functools import cached_property import tenacity +from kubernetes.client import CoreV1Api, CustomObjectsApi, models as k8s +from kubernetes.client.rest import ApiException from airflow.exceptions import AirflowException from airflow.providers.cncf.kubernetes.resource_convert.configmap import ( @@ -37,8 +39,6 @@ ) from airflow.providers.cncf.kubernetes.utils.pod_manager import PodManager from airflow.utils.log.logging_mixin import LoggingMixin -from kubernetes.client import CoreV1Api, CustomObjectsApi, models as k8s -from kubernetes.client.rest import ApiException def should_retry_start_spark_job(exception: BaseException) -> bool: diff --git a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/operators/job.py b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/operators/job.py index eb6dfae43162a..b6f06dec04276 100644 --- a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/operators/job.py +++ b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/operators/job.py @@ -26,6 +26,10 @@ from functools import cached_property from typing import TYPE_CHECKING +from kubernetes.client import BatchV1Api, models as k8s +from kubernetes.client.api_client import ApiClient +from kubernetes.client.rest import ApiException + from airflow.configuration import conf from airflow.exceptions import AirflowException from airflow.models import BaseOperator @@ -40,9 +44,6 @@ from airflow.providers.cncf.kubernetes.utils.pod_manager import EMPTY_XCOM_RESULT, PodNotFoundException from airflow.utils import yaml from airflow.utils.context import Context -from kubernetes.client import BatchV1Api, models as k8s -from kubernetes.client.api_client import ApiClient -from kubernetes.client.rest import ApiException if TYPE_CHECKING: from airflow.utils.context import Context diff --git a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/operators/kueue.py b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/operators/kueue.py index aa6ebb32ad5c5..b1c28ffcbca9a 100644 --- a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/operators/kueue.py +++ b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/operators/kueue.py @@ -22,11 +22,12 @@ from collections.abc import Sequence from functools import cached_property +from kubernetes.utils import FailToCreateError + from airflow.exceptions import AirflowException from airflow.models import BaseOperator from airflow.providers.cncf.kubernetes.hooks.kubernetes import KubernetesHook from airflow.providers.cncf.kubernetes.operators.job import KubernetesJobOperator -from kubernetes.utils import FailToCreateError class KubernetesInstallKueueOperator(BaseOperator): diff --git a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/operators/pod.py b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/operators/pod.py index 01261dfc57cdf..2eb425fcb53f7 100644 --- a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/operators/pod.py +++ b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/operators/pod.py @@ -32,10 +32,13 @@ from functools import cached_property from typing import TYPE_CHECKING, Any, Callable, Literal +import kubernetes import tenacity +from kubernetes.client import CoreV1Api, V1Pod, models as k8s +from kubernetes.client.exceptions import ApiException +from kubernetes.stream import stream from urllib3.exceptions import HTTPError -import kubernetes from airflow.configuration import conf from airflow.exceptions import ( AirflowException, @@ -81,9 +84,6 @@ from airflow.utils import yaml from airflow.utils.helpers import prune_dict, validate_key from airflow.version import version as airflow_version -from kubernetes.client import CoreV1Api, V1Pod, models as k8s -from kubernetes.client.exceptions import ApiException -from kubernetes.stream import stream if TYPE_CHECKING: import jinja2 diff --git a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/operators/resource.py b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/operators/resource.py index ca2767d48e520..aef972faf26e8 100644 --- a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/operators/resource.py +++ b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/operators/resource.py @@ -25,6 +25,7 @@ import tenacity import yaml +from kubernetes.utils import create_from_yaml from airflow.exceptions import AirflowException from airflow.models import BaseOperator @@ -32,7 +33,6 @@ from airflow.providers.cncf.kubernetes.kubernetes_helper_functions import should_retry_creation from airflow.providers.cncf.kubernetes.utils.delete_from import delete_from_yaml from airflow.providers.cncf.kubernetes.utils.k8s_resource_iterator import k8s_resource_iterator -from kubernetes.utils import create_from_yaml if TYPE_CHECKING: from kubernetes.client import ApiClient, CustomObjectsApi diff --git a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/operators/spark_kubernetes.py b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/operators/spark_kubernetes.py index d90085e68c44b..9b77a976073f6 100644 --- a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/operators/spark_kubernetes.py +++ b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/operators/spark_kubernetes.py @@ -21,6 +21,8 @@ from pathlib import Path from typing import TYPE_CHECKING, Any, cast +from kubernetes.client import CoreV1Api, CustomObjectsApi, models as k8s + from airflow.exceptions import AirflowException from airflow.providers.cncf.kubernetes import pod_generator from airflow.providers.cncf.kubernetes.hooks.kubernetes import KubernetesHook, _load_body_to_dict @@ -30,7 +32,6 @@ from airflow.providers.cncf.kubernetes.pod_generator import MAX_LABEL_LEN, PodGenerator from airflow.providers.cncf.kubernetes.utils.pod_manager import PodManager from airflow.utils.helpers import prune_dict -from kubernetes.client import CoreV1Api, CustomObjectsApi, models as k8s if TYPE_CHECKING: import jinja2 diff --git a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/pod_generator.py b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/pod_generator.py index e1f599a40a89b..b51e28c0a27f0 100644 --- a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/pod_generator.py +++ b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/pod_generator.py @@ -34,6 +34,8 @@ import re2 from dateutil import parser +from kubernetes.client import V1EmptyDirVolumeSource, V1Volume, V1VolumeMount, models as k8s +from kubernetes.client.api_client import ApiClient from airflow.exceptions import ( AirflowConfigException, @@ -47,8 +49,6 @@ from airflow.utils import yaml from airflow.utils.hashlib_wrapper import md5 from airflow.version import version as airflow_version -from kubernetes.client import V1EmptyDirVolumeSource, V1Volume, V1VolumeMount, models as k8s -from kubernetes.client.api_client import ApiClient if TYPE_CHECKING: import datetime diff --git a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/resource_convert/env_variable.py b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/resource_convert/env_variable.py index d950acf5ac42b..db8c5301cb051 100644 --- a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/resource_convert/env_variable.py +++ b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/resource_convert/env_variable.py @@ -16,9 +16,10 @@ # under the License. from __future__ import annotations -from airflow.exceptions import AirflowException from kubernetes.client import models as k8s +from airflow.exceptions import AirflowException + def convert_env_vars(env_vars) -> list[k8s.V1EnvVar]: """ diff --git a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/secret.py b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/secret.py index 2fc6c87757313..692777de447cd 100644 --- a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/secret.py +++ b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/secret.py @@ -21,9 +21,10 @@ import copy import uuid +from kubernetes.client import models as k8s + from airflow.exceptions import AirflowConfigException from airflow.providers.cncf.kubernetes.k8s_model import K8SModel -from kubernetes.client import models as k8s class Secret(K8SModel): diff --git a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/sensors/spark_kubernetes.py b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/sensors/spark_kubernetes.py index 7a42c21d7184c..08968d245fde7 100644 --- a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/sensors/spark_kubernetes.py +++ b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/sensors/spark_kubernetes.py @@ -21,10 +21,11 @@ from functools import cached_property from typing import TYPE_CHECKING +from kubernetes import client + from airflow.exceptions import AirflowException from airflow.providers.cncf.kubernetes.hooks.kubernetes import KubernetesHook from airflow.sensors.base import BaseSensorOperator -from kubernetes import client if TYPE_CHECKING: from airflow.utils.context import Context diff --git a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/template_rendering.py b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/template_rendering.py index ed2fa7e92f477..7f2cd83f33f4c 100644 --- a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/template_rendering.py +++ b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/template_rendering.py @@ -20,13 +20,13 @@ from typing import TYPE_CHECKING from jinja2 import TemplateAssertionError, UndefinedError +from kubernetes.client.api_client import ApiClient from airflow.exceptions import AirflowException from airflow.providers.cncf.kubernetes.kube_config import KubeConfig from airflow.providers.cncf.kubernetes.kubernetes_helper_functions import create_unique_id from airflow.providers.cncf.kubernetes.pod_generator import PodGenerator from airflow.utils.session import NEW_SESSION, provide_session -from kubernetes.client.api_client import ApiClient if TYPE_CHECKING: from airflow.models.taskinstance import TaskInstance diff --git a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/utils/k8s_resource_iterator.py b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/utils/k8s_resource_iterator.py index 6433ae465a40b..d66eb9fc80bc0 100644 --- a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/utils/k8s_resource_iterator.py +++ b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/utils/k8s_resource_iterator.py @@ -19,9 +19,10 @@ from collections.abc import Iterator from typing import Callable -from airflow.providers.cncf.kubernetes.utils.delete_from import FailToDeleteError from kubernetes.utils import FailToCreateError +from airflow.providers.cncf.kubernetes.utils.delete_from import FailToDeleteError + def k8s_resource_iterator(callback: Callable[[dict], None], resources: Iterator) -> None: failures: list = [] diff --git a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/utils/pod_manager.py b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/utils/pod_manager.py index 9ea5228dac040..199d6a6d35dd3 100644 --- a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/utils/pod_manager.py +++ b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/utils/pod_manager.py @@ -31,6 +31,9 @@ import pendulum import tenacity +from kubernetes import client, watch +from kubernetes.client.rest import ApiException +from kubernetes.stream import stream as kubernetes_stream from pendulum import DateTime from pendulum.parsing.exceptions import ParserError from typing_extensions import Literal @@ -41,16 +44,12 @@ from airflow.providers.cncf.kubernetes.utils.xcom_sidecar import PodDefaults from airflow.utils.log.logging_mixin import LoggingMixin from airflow.utils.timezone import utcnow -from kubernetes import client, watch -from kubernetes.client.rest import ApiException -from kubernetes.stream import stream as kubernetes_stream if TYPE_CHECKING: - from urllib3.response import HTTPResponse - from kubernetes.client.models.core_v1_event_list import CoreV1EventList from kubernetes.client.models.v1_container_status import V1ContainerStatus from kubernetes.client.models.v1_pod import V1Pod + from urllib3.response import HTTPResponse EMPTY_XCOM_RESULT = "__airflow_xcom_result_empty__" diff --git a/providers/cohere/src/airflow/__init__.py b/providers/cohere/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/cohere/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/cohere/src/airflow/providers/__init__.py b/providers/cohere/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/cohere/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/cohere/src/airflow/providers/cohere/hooks/cohere.py b/providers/cohere/src/airflow/providers/cohere/hooks/cohere.py index 1756a10dcfccd..b2d8c5d4476ff 100644 --- a/providers/cohere/src/airflow/providers/cohere/hooks/cohere.py +++ b/providers/cohere/src/airflow/providers/cohere/hooks/cohere.py @@ -23,9 +23,10 @@ from typing import TYPE_CHECKING, Any import cohere +from cohere.types import UserChatMessageV2 + from airflow.exceptions import AirflowProviderDeprecationWarning from airflow.hooks.base import BaseHook -from cohere.types import UserChatMessageV2 if TYPE_CHECKING: from cohere.core.request_options import RequestOptions diff --git a/providers/common/compat/src/airflow/__init__.py b/providers/common/compat/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/common/compat/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/common/compat/src/airflow/providers/__init__.py b/providers/common/compat/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/common/compat/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/common/compat/src/airflow/providers/common/__init__.py b/providers/common/compat/src/airflow/providers/common/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/common/compat/src/airflow/providers/common/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/common/io/src/airflow/__init__.py b/providers/common/io/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/common/io/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/common/io/src/airflow/providers/__init__.py b/providers/common/io/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/common/io/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/common/io/src/airflow/providers/common/__init__.py b/providers/common/io/src/airflow/providers/common/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/common/io/src/airflow/providers/common/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/common/sql/src/airflow/__init__.py b/providers/common/sql/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/common/sql/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/common/sql/src/airflow/providers/__init__.py b/providers/common/sql/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/common/sql/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/common/sql/src/airflow/providers/common/__init__.py b/providers/common/sql/src/airflow/providers/common/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/common/sql/src/airflow/providers/common/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/databricks/src/airflow/__init__.py b/providers/databricks/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/databricks/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/databricks/src/airflow/providers/__init__.py b/providers/databricks/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/databricks/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/databricks/src/airflow/providers/databricks/hooks/databricks_sql.py b/providers/databricks/src/airflow/providers/databricks/hooks/databricks_sql.py index 7dc15fdf5917d..b0f52798d9fc5 100644 --- a/providers/databricks/src/airflow/providers/databricks/hooks/databricks_sql.py +++ b/providers/databricks/src/airflow/providers/databricks/hooks/databricks_sql.py @@ -31,13 +31,14 @@ overload, ) +from databricks import sql # type: ignore[attr-defined] +from databricks.sql.types import Row + from airflow.exceptions import AirflowException from airflow.models.connection import Connection as AirflowConnection from airflow.providers.common.sql.hooks.sql import DbApiHook, return_single_query_results from airflow.providers.databricks.exceptions import DatabricksSqlExecutionError, DatabricksSqlExecutionTimeout from airflow.providers.databricks.hooks.databricks_base import BaseDatabricksHook -from databricks import sql # type: ignore[attr-defined] -from databricks.sql.types import Row if TYPE_CHECKING: from databricks.sql.client import Connection diff --git a/providers/databricks/src/airflow/providers/databricks/operators/databricks_sql.py b/providers/databricks/src/airflow/providers/databricks/operators/databricks_sql.py index 06a4c2f445650..a4cb062c9b8c3 100644 --- a/providers/databricks/src/airflow/providers/databricks/operators/databricks_sql.py +++ b/providers/databricks/src/airflow/providers/databricks/operators/databricks_sql.py @@ -24,11 +24,12 @@ from collections.abc import Sequence from typing import TYPE_CHECKING, Any, ClassVar +from databricks.sql.utils import ParamEscaper + from airflow.exceptions import AirflowException from airflow.models import BaseOperator from airflow.providers.common.sql.operators.sql import SQLExecuteQueryOperator from airflow.providers.databricks.hooks.databricks_sql import DatabricksSqlHook -from databricks.sql.utils import ParamEscaper if TYPE_CHECKING: from airflow.utils.context import Context diff --git a/providers/databricks/src/airflow/providers/databricks/sensors/databricks_partition.py b/providers/databricks/src/airflow/providers/databricks/sensors/databricks_partition.py index b3a10a9e9462e..df9f8cf3fe1d1 100644 --- a/providers/databricks/src/airflow/providers/databricks/sensors/databricks_partition.py +++ b/providers/databricks/src/airflow/providers/databricks/sensors/databricks_partition.py @@ -25,11 +25,12 @@ from functools import cached_property from typing import TYPE_CHECKING, Any, Callable +from databricks.sql.utils import ParamEscaper + from airflow.exceptions import AirflowException from airflow.providers.common.sql.hooks.sql import fetch_all_handler from airflow.providers.databricks.hooks.databricks_sql import DatabricksSqlHook from airflow.sensors.base import BaseSensorOperator -from databricks.sql.utils import ParamEscaper if TYPE_CHECKING: try: diff --git a/providers/datadog/src/airflow/__init__.py b/providers/datadog/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/datadog/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/datadog/src/airflow/providers/__init__.py b/providers/datadog/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/datadog/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/datadog/src/airflow/providers/datadog/hooks/datadog.py b/providers/datadog/src/airflow/providers/datadog/hooks/datadog.py index ba0cf9286e664..3530aa7677ec4 100644 --- a/providers/datadog/src/airflow/providers/datadog/hooks/datadog.py +++ b/providers/datadog/src/airflow/providers/datadog/hooks/datadog.py @@ -20,10 +20,11 @@ import time from typing import Any +from datadog import api, initialize # type: ignore[attr-defined] + from airflow.exceptions import AirflowException from airflow.hooks.base import BaseHook from airflow.utils.log.logging_mixin import LoggingMixin -from datadog import api, initialize # type: ignore[attr-defined] class DatadogHook(BaseHook, LoggingMixin): diff --git a/providers/datadog/src/airflow/providers/datadog/sensors/datadog.py b/providers/datadog/src/airflow/providers/datadog/sensors/datadog.py index 6e5b4c9e08aed..bf74e5fd86481 100644 --- a/providers/datadog/src/airflow/providers/datadog/sensors/datadog.py +++ b/providers/datadog/src/airflow/providers/datadog/sensors/datadog.py @@ -19,10 +19,11 @@ from typing import TYPE_CHECKING, Any, Callable +from datadog import api + from airflow.exceptions import AirflowException from airflow.providers.datadog.hooks.datadog import DatadogHook from airflow.sensors.base import BaseSensorOperator -from datadog import api if TYPE_CHECKING: try: diff --git a/providers/dbt/cloud/src/airflow/__init__.py b/providers/dbt/cloud/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/dbt/cloud/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/dbt/cloud/src/airflow/providers/__init__.py b/providers/dbt/cloud/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/dbt/cloud/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/dbt/cloud/src/airflow/providers/dbt/__init__.py b/providers/dbt/cloud/src/airflow/providers/dbt/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/dbt/cloud/src/airflow/providers/dbt/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/dingding/src/airflow/__init__.py b/providers/dingding/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/dingding/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/dingding/src/airflow/providers/__init__.py b/providers/dingding/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/dingding/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/discord/src/airflow/__init__.py b/providers/discord/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/discord/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/discord/src/airflow/providers/__init__.py b/providers/discord/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/discord/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/docker/src/airflow/__init__.py b/providers/docker/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/docker/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/docker/src/airflow/providers/__init__.py b/providers/docker/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/docker/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/docker/src/airflow/providers/docker/hooks/docker.py b/providers/docker/src/airflow/providers/docker/hooks/docker.py index 6da06b7e9ae33..fa1377bedd021 100644 --- a/providers/docker/src/airflow/providers/docker/hooks/docker.py +++ b/providers/docker/src/airflow/providers/docker/hooks/docker.py @@ -22,12 +22,13 @@ from functools import cached_property from typing import TYPE_CHECKING, Any -from airflow.exceptions import AirflowException, AirflowNotFoundException -from airflow.hooks.base import BaseHook from docker import APIClient, TLSConfig from docker.constants import DEFAULT_TIMEOUT_SECONDS from docker.errors import APIError, DockerException +from airflow.exceptions import AirflowException, AirflowNotFoundException +from airflow.hooks.base import BaseHook + if TYPE_CHECKING: from airflow.models import Connection diff --git a/providers/docker/src/airflow/providers/docker/operators/docker.py b/providers/docker/src/airflow/providers/docker/operators/docker.py index dd9bd630bec2b..a314282adcd39 100644 --- a/providers/docker/src/airflow/providers/docker/operators/docker.py +++ b/providers/docker/src/airflow/providers/docker/operators/docker.py @@ -29,6 +29,9 @@ from tempfile import TemporaryDirectory from typing import TYPE_CHECKING +from docker.constants import DEFAULT_TIMEOUT_SECONDS +from docker.errors import APIError +from docker.types import LogConfig, Mount, Ulimit from dotenv import dotenv_values from typing_extensions import Literal @@ -38,9 +41,6 @@ DockerContainerFailedSkipException, ) from airflow.providers.docker.hooks.docker import DockerHook -from docker.constants import DEFAULT_TIMEOUT_SECONDS -from docker.errors import APIError -from docker.types import LogConfig, Mount, Ulimit if TYPE_CHECKING: from logging import Logger diff --git a/providers/docker/src/airflow/providers/docker/operators/docker_swarm.py b/providers/docker/src/airflow/providers/docker/operators/docker_swarm.py index 200c03b127694..6a77f73b81055 100644 --- a/providers/docker/src/airflow/providers/docker/operators/docker_swarm.py +++ b/providers/docker/src/airflow/providers/docker/operators/docker_swarm.py @@ -24,11 +24,12 @@ from time import sleep from typing import TYPE_CHECKING, Literal +from docker import types +from docker.errors import APIError + from airflow.exceptions import AirflowException from airflow.providers.docker.operators.docker import DockerOperator from airflow.utils.strings import get_random_string -from docker import types -from docker.errors import APIError if TYPE_CHECKING: try: diff --git a/providers/edge/src/airflow/__init__.py b/providers/edge/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/edge/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/edge/src/airflow/providers/__init__.py b/providers/edge/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/edge/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/elasticsearch/src/airflow/__init__.py b/providers/elasticsearch/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/elasticsearch/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/elasticsearch/src/airflow/providers/__init__.py b/providers/elasticsearch/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/elasticsearch/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/elasticsearch/src/airflow/providers/elasticsearch/hooks/elasticsearch.py b/providers/elasticsearch/src/airflow/providers/elasticsearch/hooks/elasticsearch.py index 995d87415bb37..758a2e5dabc6b 100644 --- a/providers/elasticsearch/src/airflow/providers/elasticsearch/hooks/elasticsearch.py +++ b/providers/elasticsearch/src/airflow/providers/elasticsearch/hooks/elasticsearch.py @@ -22,9 +22,10 @@ from typing import TYPE_CHECKING, Any from urllib import parse +from elasticsearch import Elasticsearch + from airflow.hooks.base import BaseHook from airflow.providers.common.sql.hooks.sql import DbApiHook -from elasticsearch import Elasticsearch if TYPE_CHECKING: from elastic_transport import ObjectApiResponse diff --git a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py index 5343ba46618ea..15904e7ebf3b4 100644 --- a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py +++ b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py @@ -31,10 +31,12 @@ from typing import TYPE_CHECKING, Any, Callable, Literal from urllib.parse import quote, urlparse -import pendulum - # Using `from elasticsearch import *` would break elasticsearch mocking used in unit test. import elasticsearch +import pendulum +from elasticsearch import helpers +from elasticsearch.exceptions import NotFoundError + from airflow.configuration import conf from airflow.exceptions import AirflowException from airflow.models.dagrun import DagRun @@ -46,8 +48,6 @@ from airflow.utils.log.logging_mixin import ExternalLoggingMixin, LoggingMixin from airflow.utils.module_loading import import_string from airflow.utils.session import create_session -from elasticsearch import helpers -from elasticsearch.exceptions import NotFoundError if TYPE_CHECKING: from datetime import datetime diff --git a/providers/exasol/src/airflow/__init__.py b/providers/exasol/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/exasol/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/exasol/src/airflow/providers/__init__.py b/providers/exasol/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/exasol/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/fab/src/airflow/__init__.py b/providers/fab/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/fab/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/fab/src/airflow/providers/3rd-party-licenses/LICENSES-ui.txt b/providers/fab/src/airflow/providers/3rd-party-licenses/LICENSES-ui.txt deleted file mode 100644 index 7ad85fd17468f..0000000000000 --- a/providers/fab/src/airflow/providers/3rd-party-licenses/LICENSES-ui.txt +++ /dev/null @@ -1,89 +0,0 @@ -Apache Airflow -Copyright 2016-2023 The Apache Software Foundation - -This product includes software developed at The Apache Software -Foundation (http://www.apache.org/). - -======================================================================= -css-loader|5.2.7: ------ -MIT -Copyright JS Foundation and other contributors - -Permission is hereby granted, free of charge, to any person obtaining -a copy of this software and associated documentation files (the -'Software'), to deal in the Software without restriction, including -without limitation the rights to use, copy, modify, merge, publish, -distribute, sublicense, and/or sell copies of the Software, and to -permit persons to whom the Software is furnished to do so, subject to -the following conditions: - -The above copyright notice and this permission notice shall be -included in all copies or substantial portions of the Software. - -THE SOFTWARE IS PROVIDED 'AS IS', WITHOUT WARRANTY OF ANY KIND, -EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF -MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. -IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY -CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, -TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE -SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. - -webpack-contrib/css-loader - - -moment|2.30.1: ------ -MIT -Copyright (c) JS Foundation and other contributors - -Permission is hereby granted, free of charge, to any person -obtaining a copy of this software and associated documentation -files (the "Software"), to deal in the Software without -restriction, including without limitation the rights to use, -copy, modify, merge, publish, distribute, sublicense, and/or sell -copies of the Software, and to permit persons to whom the -Software is furnished to do so, subject to the following -conditions: - -The above copyright notice and this permission notice shall be -included in all copies or substantial portions of the Software. - -THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, -EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES -OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND -NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT -HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, -WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING -FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR -OTHER DEALINGS IN THE SOFTWARE. - -https://github.com/moment/moment.git - - -moment-timezone|0.5.47: ------ -MIT -The MIT License (MIT) - -Copyright (c) JS Foundation and other contributors - -Permission is hereby granted, free of charge, to any person obtaining a copy of -this software and associated documentation files (the "Software"), to deal in -the Software without restriction, including without limitation the rights to -use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of -the Software, and to permit persons to whom the Software is furnished to do so, -subject to the following conditions: - -The above copyright notice and this permission notice shall be included in all -copies or substantial portions of the Software. - -THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS -FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR -COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER -IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN -CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. -https://github.com/moment/moment-timezone.git - - diff --git a/providers/fab/src/airflow/providers/__init__.py b/providers/fab/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/fab/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/facebook/src/airflow/__init__.py b/providers/facebook/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/facebook/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/facebook/src/airflow/providers/__init__.py b/providers/facebook/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/facebook/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/ftp/src/airflow/__init__.py b/providers/ftp/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/ftp/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/ftp/src/airflow/providers/__init__.py b/providers/ftp/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/ftp/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/github/src/airflow/__init__.py b/providers/github/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/github/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/github/src/airflow/providers/__init__.py b/providers/github/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/github/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/github/src/airflow/providers/github/hooks/github.py b/providers/github/src/airflow/providers/github/hooks/github.py index fb2b2b0416008..6be50fd31fec4 100644 --- a/providers/github/src/airflow/providers/github/hooks/github.py +++ b/providers/github/src/airflow/providers/github/hooks/github.py @@ -21,9 +21,10 @@ from typing import TYPE_CHECKING +from github import Github as GithubClient + from airflow.exceptions import AirflowException from airflow.hooks.base import BaseHook -from github import Github as GithubClient class GithubHook(BaseHook): diff --git a/providers/github/src/airflow/providers/github/operators/github.py b/providers/github/src/airflow/providers/github/operators/github.py index 3889335628d03..82996d3ecedb4 100644 --- a/providers/github/src/airflow/providers/github/operators/github.py +++ b/providers/github/src/airflow/providers/github/operators/github.py @@ -19,10 +19,11 @@ from typing import TYPE_CHECKING, Any, Callable +from github import GithubException + from airflow.exceptions import AirflowException from airflow.models import BaseOperator from airflow.providers.github.hooks.github import GithubHook -from github import GithubException if TYPE_CHECKING: try: diff --git a/providers/github/src/airflow/providers/github/sensors/github.py b/providers/github/src/airflow/providers/github/sensors/github.py index b40420ce207ab..cacaef9e32fb8 100644 --- a/providers/github/src/airflow/providers/github/sensors/github.py +++ b/providers/github/src/airflow/providers/github/sensors/github.py @@ -19,10 +19,11 @@ from typing import TYPE_CHECKING, Any, Callable +from github import GithubException + from airflow.exceptions import AirflowException from airflow.providers.github.hooks.github import GithubHook from airflow.sensors.base import BaseSensorOperator -from github import GithubException if TYPE_CHECKING: try: diff --git a/providers/google/src/airflow/__init__.py b/providers/google/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/google/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/google/src/airflow/providers/__init__.py b/providers/google/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/google/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/google/src/airflow/providers/google/ads/hooks/ads.py b/providers/google/src/airflow/providers/google/ads/hooks/ads.py index 97d61dce325e2..c1d35a613a2f1 100644 --- a/providers/google/src/airflow/providers/google/ads/hooks/ads.py +++ b/providers/google/src/airflow/providers/google/ads/hooks/ads.py @@ -24,13 +24,14 @@ from tempfile import NamedTemporaryFile from typing import IO, TYPE_CHECKING, Any, Literal -from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning -from airflow.hooks.base import BaseHook -from airflow.providers.google.common.hooks.base_google import get_field from google.ads.googleads.client import GoogleAdsClient from google.ads.googleads.errors import GoogleAdsException from google.auth.exceptions import GoogleAuthError +from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning +from airflow.hooks.base import BaseHook +from airflow.providers.google.common.hooks.base_google import get_field + if TYPE_CHECKING: from google.ads.googleads.v18.services.services.customer_service import CustomerServiceClient from google.ads.googleads.v18.services.services.google_ads_service import GoogleAdsServiceClient diff --git a/providers/google/src/airflow/providers/google/cloud/_internal_client/secret_manager_client.py b/providers/google/src/airflow/providers/google/cloud/_internal_client/secret_manager_client.py index 09ca5ec210bb8..a78d0c7bdb237 100644 --- a/providers/google/src/airflow/providers/google/cloud/_internal_client/secret_manager_client.py +++ b/providers/google/src/airflow/providers/google/cloud/_internal_client/secret_manager_client.py @@ -20,11 +20,12 @@ from functools import cached_property from typing import TYPE_CHECKING -from airflow.providers.google.common.consts import CLIENT_INFO -from airflow.utils.log.logging_mixin import LoggingMixin from google.api_core.exceptions import InvalidArgument, NotFound, PermissionDenied from google.cloud.secretmanager_v1 import SecretManagerServiceClient +from airflow.providers.google.common.consts import CLIENT_INFO +from airflow.utils.log.logging_mixin import LoggingMixin + if TYPE_CHECKING: import google diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/alloy_db.py b/providers/google/src/airflow/providers/google/cloud/hooks/alloy_db.py index 72c7846258d74..38a1b622e84fa 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/alloy_db.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/alloy_db.py @@ -24,16 +24,15 @@ from typing import TYPE_CHECKING import tenacity +from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault +from google.cloud import alloydb_v1 from airflow.exceptions import AirflowException from airflow.providers.google.common.consts import CLIENT_INFO from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID, GoogleBaseHook -from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault -from google.cloud import alloydb_v1 if TYPE_CHECKING: import proto - from google.api_core.operation import Operation from google.api_core.retry import Retry from google.protobuf.field_mask_pb2 import FieldMask diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/automl.py b/providers/google/src/airflow/providers/google/cloud/hooks/automl.py index 27d6be832f305..ba13b1e4edb77 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/automl.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/automl.py @@ -29,10 +29,6 @@ from functools import cached_property from typing import TYPE_CHECKING -from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning -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 google.api_core.gapic_v1.method import DEFAULT, _MethodDefault from google.cloud.automl_v1beta1 import ( AutoMlClient, @@ -47,6 +43,11 @@ PredictResponse, ) +from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning +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 + if TYPE_CHECKING: from google.api_core.operation import Operation from google.api_core.retry import Retry diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/bigquery.py b/providers/google/src/airflow/providers/google/cloud/hooks/bigquery.py index 3ff99367d30f6..ff2ee2eb1d8f2 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/bigquery.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/bigquery.py @@ -33,6 +33,26 @@ from aiohttp import ClientSession as ClientSession from gcloud.aio.bigquery import Job, Table as Table_async +from google.cloud.bigquery import ( + DEFAULT_RETRY, + Client, + CopyJob, + ExtractJob, + LoadJob, + QueryJob, + SchemaField, + UnknownJob, +) +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 ( + Row, + RowIterator, + Table, + TableListItem, + TableReference, +) +from google.cloud.exceptions import NotFound 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`` @@ -55,30 +75,9 @@ from airflow.utils.hashlib_wrapper import md5 from airflow.utils.helpers import convert_camel_to_snake from airflow.utils.log.logging_mixin import LoggingMixin -from google.cloud.bigquery import ( - DEFAULT_RETRY, - Client, - CopyJob, - ExtractJob, - LoadJob, - QueryJob, - SchemaField, - UnknownJob, -) -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 ( - Row, - RowIterator, - Table, - TableListItem, - TableReference, -) -from google.cloud.exceptions import NotFound if TYPE_CHECKING: import pandas as pd - from google.api_core.page_iterator import HTTPIterator from google.api_core.retry import Retry diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/bigquery_dts.py b/providers/google/src/airflow/providers/google/cloud/hooks/bigquery_dts.py index a5f35b5bff48a..467624778082c 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/bigquery_dts.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/bigquery_dts.py @@ -23,12 +23,6 @@ from copy import copy from typing import TYPE_CHECKING -from airflow.providers.google.common.consts import CLIENT_INFO -from airflow.providers.google.common.hooks.base_google import ( - PROVIDE_PROJECT_ID, - GoogleBaseAsyncHook, - GoogleBaseHook, -) from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault from google.cloud.bigquery_datatransfer_v1 import DataTransferServiceAsyncClient, DataTransferServiceClient from google.cloud.bigquery_datatransfer_v1.types import ( @@ -37,11 +31,17 @@ TransferRun, ) -if TYPE_CHECKING: - from googleapiclient.discovery import Resource +from airflow.providers.google.common.consts import CLIENT_INFO +from airflow.providers.google.common.hooks.base_google import ( + PROVIDE_PROJECT_ID, + GoogleBaseAsyncHook, + GoogleBaseHook, +) +if TYPE_CHECKING: from google.api_core.retry import Retry from google.api_core.retry_async import AsyncRetry + from googleapiclient.discovery import Resource def get_object_id(obj: dict) -> str: diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/bigtable.py b/providers/google/src/airflow/providers/google/cloud/hooks/bigtable.py index 860c46eb0ee0b..83a631eafa687 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/bigtable.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/bigtable.py @@ -22,13 +22,14 @@ from collections.abc import Sequence from typing import TYPE_CHECKING -from airflow.providers.google.common.consts import CLIENT_INFO -from airflow.providers.google.common.hooks.base_google import GoogleBaseHook from google.cloud.bigtable import Client, enums from google.cloud.bigtable.cluster import Cluster from google.cloud.bigtable.instance import Instance from google.cloud.bigtable.table import ClusterState, Table +from airflow.providers.google.common.consts import CLIENT_INFO +from airflow.providers.google.common.hooks.base_google import GoogleBaseHook + if TYPE_CHECKING: import enum diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/cloud_batch.py b/providers/google/src/airflow/providers/google/cloud/hooks/cloud_batch.py index d860046f970f7..c647ac078abb2 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/cloud_batch.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/cloud_batch.py @@ -23,9 +23,6 @@ from collections.abc import Iterable, Sequence from typing import TYPE_CHECKING -from airflow.exceptions import AirflowException -from airflow.providers.google.common.consts import CLIENT_INFO -from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID, GoogleBaseHook from google.cloud.batch import ListJobsRequest, ListTasksRequest from google.cloud.batch_v1 import ( BatchServiceAsyncClient, @@ -36,6 +33,10 @@ Task, ) +from airflow.exceptions import AirflowException +from airflow.providers.google.common.consts import CLIENT_INFO +from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID, GoogleBaseHook + if TYPE_CHECKING: from google.api_core import operation from google.cloud.batch_v1.services.batch_service import pagers diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/cloud_build.py b/providers/google/src/airflow/providers/google/cloud/hooks/cloud_build.py index 924c9a977ae98..ad28f22df2fd3 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/cloud_build.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/cloud_build.py @@ -22,15 +22,16 @@ from collections.abc import Sequence from typing import TYPE_CHECKING -from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning -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 google.api_core.client_options import ClientOptions from google.api_core.exceptions import AlreadyExists from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault from google.cloud.devtools.cloudbuild_v1 import CloudBuildAsyncClient, CloudBuildClient, GetBuildRequest +from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning +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 + if TYPE_CHECKING: from google.api_core.operation import Operation from google.api_core.retry import Retry diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/cloud_composer.py b/providers/google/src/airflow/providers/google/cloud/hooks/cloud_composer.py index a748d53c8b69e..de71d5af8212a 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/cloud_composer.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/cloud_composer.py @@ -22,9 +22,6 @@ from collections.abc import MutableSequence, Sequence from typing import TYPE_CHECKING -from airflow.exceptions import AirflowException -from airflow.providers.google.common.consts import CLIENT_INFO -from airflow.providers.google.common.hooks.base_google import GoogleBaseHook from google.api_core.client_options import ClientOptions from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault from google.cloud.orchestration.airflow.service_v1 import ( @@ -34,6 +31,10 @@ PollAirflowCommandResponse, ) +from airflow.exceptions import AirflowException +from airflow.providers.google.common.consts import CLIENT_INFO +from airflow.providers.google.common.hooks.base_google import GoogleBaseHook + if TYPE_CHECKING: from google.api_core.operation import Operation from google.api_core.operation_async import AsyncOperation diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/cloud_memorystore.py b/providers/google/src/airflow/providers/google/cloud/hooks/cloud_memorystore.py index d4f010885dd43..eb75e65b42e64 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/cloud_memorystore.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/cloud_memorystore.py @@ -31,9 +31,6 @@ from collections.abc import Sequence from typing import TYPE_CHECKING -from airflow import version -from airflow.exceptions import AirflowException -from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID, GoogleBaseHook from google.api_core import path_template from google.api_core.exceptions import NotFound from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault @@ -47,6 +44,10 @@ OutputConfig, ) +from airflow import version +from airflow.exceptions import AirflowException +from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID, GoogleBaseHook + if TYPE_CHECKING: from google.api_core.retry import Retry from google.protobuf.field_mask_pb2 import FieldMask diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/cloud_run.py b/providers/google/src/airflow/providers/google/cloud/hooks/cloud_run.py index 8b19182dd2f1f..af8fc8d96aa66 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/cloud_run.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/cloud_run.py @@ -21,9 +21,6 @@ from collections.abc import Iterable, Sequence from typing import TYPE_CHECKING, Any -from airflow.exceptions import AirflowException -from airflow.providers.google.common.consts import CLIENT_INFO -from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID, GoogleBaseHook from google.cloud.run_v2 import ( CreateJobRequest, CreateServiceRequest, @@ -43,6 +40,10 @@ ) from google.longrunning import operations_pb2 # type: ignore[attr-defined] +from airflow.exceptions import AirflowException +from airflow.providers.google.common.consts import CLIENT_INFO +from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID, GoogleBaseHook + if TYPE_CHECKING: from google.api_core import operation from google.api_core.operation_async import AsyncOperation diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/cloud_sql.py b/providers/google/src/airflow/providers/google/cloud/hooks/cloud_sql.py index 302a2ce7638a5..bc6dedf459868 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/cloud_sql.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/cloud_sql.py @@ -64,10 +64,10 @@ from airflow.utils.log.logging_mixin import LoggingMixin if TYPE_CHECKING: + from google.cloud.secretmanager_v1 import AccessSecretVersionResponse from requests import Session from airflow.providers.common.sql.hooks.sql import DbApiHook - from google.cloud.secretmanager_v1 import AccessSecretVersionResponse UNIX_PATH_MAX = 108 diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/cloud_storage_transfer_service.py b/providers/google/src/airflow/providers/google/cloud/hooks/cloud_storage_transfer_service.py index 6da2c57ae95a5..b9d19e81e174f 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/cloud_storage_transfer_service.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/cloud_storage_transfer_service.py @@ -36,6 +36,13 @@ from datetime import timedelta from typing import TYPE_CHECKING, Any +from google.api_core import protobuf_helpers +from google.cloud.storage_transfer_v1 import ( + ListTransferJobsRequest, + StorageTransferServiceAsyncClient, + TransferJob, + TransferOperation, +) from googleapiclient.discovery import Resource, build from googleapiclient.errors import HttpError @@ -46,21 +53,13 @@ GoogleBaseAsyncHook, GoogleBaseHook, ) -from google.api_core import protobuf_helpers -from google.cloud.storage_transfer_v1 import ( - ListTransferJobsRequest, - StorageTransferServiceAsyncClient, - TransferJob, - TransferOperation, -) if TYPE_CHECKING: - from proto import Message - from google.cloud.storage_transfer_v1.services.storage_transfer_service.pagers import ( ListTransferJobsAsyncPager, ) from google.longrunning import operations_pb2 # type: ignore[attr-defined] + from proto import Message log = logging.getLogger(__name__) diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/compute.py b/providers/google/src/airflow/providers/google/cloud/hooks/compute.py index f5a1bfd836a8d..bb86fbfa25547 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/compute.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/compute.py @@ -23,13 +23,13 @@ from collections.abc import Sequence from typing import TYPE_CHECKING, Any +from google.cloud.compute_v1.services.instance_group_managers import InstanceGroupManagersClient +from google.cloud.compute_v1.services.instance_templates import InstanceTemplatesClient +from google.cloud.compute_v1.services.instances import InstancesClient from googleapiclient.discovery import build from airflow.exceptions import AirflowException from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID, GoogleBaseHook -from google.cloud.compute_v1.services.instance_group_managers import InstanceGroupManagersClient -from google.cloud.compute_v1.services.instance_templates import InstanceTemplatesClient -from google.cloud.compute_v1.services.instances import InstancesClient if TYPE_CHECKING: from google.api_core.retry import Retry diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/datacatalog.py b/providers/google/src/airflow/providers/google/cloud/hooks/datacatalog.py index fc1dc0262f2ca..2cfa308ff9248 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/datacatalog.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/datacatalog.py @@ -19,9 +19,6 @@ from collections.abc import Sequence from typing import TYPE_CHECKING -from airflow.exceptions import AirflowException -from airflow.providers.google.common.consts import CLIENT_INFO -from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID, GoogleBaseHook from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault from google.cloud import datacatalog from google.cloud.datacatalog import ( @@ -35,6 +32,10 @@ TagTemplateField, ) +from airflow.exceptions import AirflowException +from airflow.providers.google.common.consts import CLIENT_INFO +from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID, GoogleBaseHook + if TYPE_CHECKING: from google.api_core.retry import Retry from google.protobuf.field_mask_pb2 import FieldMask diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/dataflow.py b/providers/google/src/airflow/providers/google/cloud/hooks/dataflow.py index 1e08bb694f11e..ab26ccc38d61c 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/dataflow.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/dataflow.py @@ -31,18 +31,6 @@ from copy import deepcopy from typing import TYPE_CHECKING, Any, Callable, TypeVar, cast -from googleapiclient.discovery import Resource, build - -from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning -from airflow.providers.apache.beam.hooks.beam import BeamHook, BeamRunnerType, beam_options_to_args -from airflow.providers.google.common.deprecated import deprecated -from airflow.providers.google.common.hooks.base_google import ( - PROVIDE_PROJECT_ID, - GoogleBaseAsyncHook, - GoogleBaseHook, -) -from airflow.utils.log.logging_mixin import LoggingMixin -from airflow.utils.timeout import timeout from google.cloud.dataflow_v1beta3 import ( GetJobRequest, Job, @@ -59,6 +47,18 @@ JobMetrics, ) from google.cloud.dataflow_v1beta3.types.jobs import ListJobsRequest +from googleapiclient.discovery import Resource, build + +from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning +from airflow.providers.apache.beam.hooks.beam import BeamHook, BeamRunnerType, beam_options_to_args +from airflow.providers.google.common.deprecated import deprecated +from airflow.providers.google.common.hooks.base_google import ( + PROVIDE_PROJECT_ID, + GoogleBaseAsyncHook, + GoogleBaseHook, +) +from airflow.utils.log.logging_mixin import LoggingMixin +from airflow.utils.timeout import timeout if TYPE_CHECKING: from google.cloud.dataflow_v1beta3.services.jobs_v1_beta3.pagers import ListJobsAsyncPager diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/dataform.py b/providers/google/src/airflow/providers/google/cloud/hooks/dataform.py index 9ecec4ba3b51d..c921b56c6fe82 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/dataform.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/dataform.py @@ -20,8 +20,6 @@ from collections.abc import Sequence from typing import TYPE_CHECKING -from airflow.exceptions import AirflowException -from airflow.providers.google.common.hooks.base_google import GoogleBaseHook from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault from google.cloud.dataform_v1beta1 import DataformClient from google.cloud.dataform_v1beta1.types import ( @@ -33,6 +31,9 @@ WriteFileResponse, ) +from airflow.exceptions import AirflowException +from airflow.providers.google.common.hooks.base_google import GoogleBaseHook + if TYPE_CHECKING: from google.api_core.retry import Retry from google.cloud.dataform_v1beta1.services.dataform.pagers import QueryWorkflowInvocationActionsPager diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/datafusion.py b/providers/google/src/airflow/providers/google/cloud/hooks/datafusion.py index c69563a874351..89a4986adfd5c 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/datafusion.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/datafusion.py @@ -26,11 +26,12 @@ from typing import Any from urllib.parse import quote, urlencode, urljoin +import google.auth from aiohttp import ClientSession from gcloud.aio.auth import AioSession, Token +from google.api_core.retry import exponential_sleep_generator from googleapiclient.discovery import Resource, build -import google.auth from airflow.exceptions import AirflowException, AirflowNotFoundException from airflow.providers.google.cloud.utils.datafusion import DataFusionPipelineType from airflow.providers.google.common.hooks.base_google import ( @@ -38,7 +39,6 @@ GoogleBaseAsyncHook, GoogleBaseHook, ) -from google.api_core.retry import exponential_sleep_generator Operation = dict[str, Any] diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/dataplex.py b/providers/google/src/airflow/providers/google/cloud/hooks/dataplex.py index ecb75c9eccc4c..12670376fc234 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/dataplex.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/dataplex.py @@ -23,13 +23,6 @@ from copy import deepcopy from typing import TYPE_CHECKING, Any -from airflow.exceptions import AirflowException -from airflow.providers.google.common.consts import CLIENT_INFO -from airflow.providers.google.common.hooks.base_google import ( - PROVIDE_PROJECT_ID, - GoogleBaseAsyncHook, - GoogleBaseHook, -) from google.api_core.client_options import ClientOptions from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault from google.cloud.dataplex_v1 import ( @@ -53,9 +46,15 @@ ) from google.protobuf.field_mask_pb2 import FieldMask -if TYPE_CHECKING: - from googleapiclient.discovery import Resource +from airflow.exceptions import AirflowException +from airflow.providers.google.common.consts import CLIENT_INFO +from airflow.providers.google.common.hooks.base_google import ( + PROVIDE_PROJECT_ID, + GoogleBaseAsyncHook, + GoogleBaseHook, +) +if TYPE_CHECKING: from google.api_core.operation import Operation from google.api_core.retry import Retry from google.api_core.retry_async import AsyncRetry @@ -66,6 +65,7 @@ ListEntryTypesPager, SearchEntriesPager, ) + from googleapiclient.discovery import Resource PATH_DATA_SCAN = "projects/{project_id}/locations/{region}/dataScans/{data_scan_id}" diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/dataproc.py b/providers/google/src/airflow/providers/google/cloud/hooks/dataproc.py index 2f1025aac509d..6cc57128bab4b 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/dataproc.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/dataproc.py @@ -26,10 +26,6 @@ from collections.abc import MutableSequence, Sequence from typing import TYPE_CHECKING, Any -from airflow.exceptions import AirflowException -from airflow.providers.google.common.consts import CLIENT_INFO -from airflow.providers.google.common.hooks.base_google import GoogleBaseHook -from airflow.version import version as airflow_version from google.api_core.client_options import ClientOptions from google.api_core.exceptions import ServerError from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault @@ -49,6 +45,11 @@ WorkflowTemplateServiceClient, ) +from airflow.exceptions import AirflowException +from airflow.providers.google.common.consts import CLIENT_INFO +from airflow.providers.google.common.hooks.base_google import GoogleBaseHook +from airflow.version import version as airflow_version + if TYPE_CHECKING: from google.api_core.operation import Operation from google.api_core.operation_async import AsyncOperation diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/dataproc_metastore.py b/providers/google/src/airflow/providers/google/cloud/hooks/dataproc_metastore.py index 32ac81d319e53..3520397984812 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/dataproc_metastore.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/dataproc_metastore.py @@ -22,13 +22,14 @@ from collections.abc import Sequence from typing import TYPE_CHECKING, Any -from airflow.exceptions import AirflowException -from airflow.providers.google.common.consts import CLIENT_INFO -from airflow.providers.google.common.hooks.base_google import GoogleBaseHook from google.api_core.client_options import ClientOptions from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault from google.cloud.metastore_v1 import DataprocMetastoreClient +from airflow.exceptions import AirflowException +from airflow.providers.google.common.consts import CLIENT_INFO +from airflow.providers.google.common.hooks.base_google import GoogleBaseHook + if TYPE_CHECKING: from google.api_core.operation import Operation from google.api_core.retry import Retry diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/dlp.py b/providers/google/src/airflow/providers/google/cloud/hooks/dlp.py index b2a8e11f74d2d..c0730497e904e 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/dlp.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/dlp.py @@ -31,9 +31,6 @@ from collections.abc import Sequence from typing import TYPE_CHECKING -from airflow.exceptions import AirflowException -from airflow.providers.google.common.consts import CLIENT_INFO -from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID, GoogleBaseHook from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault from google.cloud.dlp import DlpServiceClient from google.cloud.dlp_v2.types import ( @@ -58,6 +55,10 @@ ) from google.protobuf.field_mask_pb2 import FieldMask +from airflow.exceptions import AirflowException +from airflow.providers.google.common.consts import CLIENT_INFO +from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID, GoogleBaseHook + if TYPE_CHECKING: from google.api_core.retry import Retry diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/gcs.py b/providers/google/src/airflow/providers/google/cloud/hooks/gcs.py index 116f5c4501ff7..3c17305a60a44 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/gcs.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/gcs.py @@ -35,6 +35,12 @@ from urllib.parse import urlsplit from gcloud.aio.storage import Storage +from google.api_core.exceptions import GoogleAPICallError, NotFound + +# not sure why but mypy complains on missing `storage` but it is clearly there and is importable +from google.cloud import storage # type: ignore[attr-defined] +from google.cloud.exceptions import GoogleCloudError +from google.cloud.storage.retry import DEFAULT_RETRY from requests import Session from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning @@ -49,18 +55,11 @@ from airflow.typing_compat import ParamSpec from airflow.utils import timezone from airflow.version import version -from google.api_core.exceptions import GoogleAPICallError, NotFound - -# not sure why but mypy complains on missing `storage` but it is clearly there and is importable -from google.cloud import storage # type: ignore[attr-defined] -from google.cloud.exceptions import GoogleCloudError -from google.cloud.storage.retry import DEFAULT_RETRY if TYPE_CHECKING: from datetime import datetime from aiohttp import ClientSession - from google.api_core.retry import Retry from google.cloud.storage.blob import Blob diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/kms.py b/providers/google/src/airflow/providers/google/cloud/hooks/kms.py index 6353cd397bfc3..e69273d993934 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/kms.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/kms.py @@ -23,11 +23,12 @@ from collections.abc import Sequence from typing import TYPE_CHECKING -from airflow.providers.google.common.consts import CLIENT_INFO -from airflow.providers.google.common.hooks.base_google import GoogleBaseHook from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault from google.cloud.kms_v1 import KeyManagementServiceClient +from airflow.providers.google.common.consts import CLIENT_INFO +from airflow.providers.google.common.hooks.base_google import GoogleBaseHook + if TYPE_CHECKING: from google.api_core.retry import Retry diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/kubernetes_engine.py b/providers/google/src/airflow/providers/google/cloud/hooks/kubernetes_engine.py index 0295f5af4e36b..3b791c5359d5c 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/kubernetes_engine.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/kubernetes_engine.py @@ -25,6 +25,14 @@ from collections.abc import Sequence from typing import TYPE_CHECKING, Any +from google.api_core.exceptions import NotFound +from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault +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 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 from kubernetes_asyncio import client as async_client from kubernetes_asyncio.config.kube_config import FileOrData @@ -39,14 +47,6 @@ GoogleBaseAsyncHook, GoogleBaseHook, ) -from google.api_core.exceptions import NotFound -from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault -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 exceptions # type: ignore[attr-defined] -from google.cloud.container_v1 import ClusterManagerAsyncClient, ClusterManagerClient -from google.cloud.container_v1.types import Cluster, Operation if TYPE_CHECKING: import google.auth.credentials diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/life_sciences.py b/providers/google/src/airflow/providers/google/cloud/hooks/life_sciences.py index d8ce45cef95cf..a3552db5c6b8f 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/life_sciences.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/life_sciences.py @@ -22,9 +22,9 @@ import time from collections.abc import Sequence +import google.api_core.path_template from googleapiclient.discovery import build -import google.api_core.path_template from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning from airflow.providers.google.common.deprecated import deprecated from airflow.providers.google.common.hooks.base_google import GoogleBaseHook diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/managed_kafka.py b/providers/google/src/airflow/providers/google/cloud/hooks/managed_kafka.py index aec8d92f997a4..f71e8a158c83f 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/managed_kafka.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/managed_kafka.py @@ -23,11 +23,12 @@ from copy import deepcopy from typing import TYPE_CHECKING +from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault +from google.cloud.managedkafka_v1 import Cluster, ManagedKafkaClient, Topic, types + from airflow.exceptions import AirflowException from airflow.providers.google.common.consts import CLIENT_INFO from airflow.providers.google.common.hooks.base_google import GoogleBaseHook -from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault -from google.cloud.managedkafka_v1 import Cluster, ManagedKafkaClient, Topic, types if TYPE_CHECKING: from google.api_core.operation import Operation diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/natural_language.py b/providers/google/src/airflow/providers/google/cloud/hooks/natural_language.py index d1f24b6a2adce..edeef2d393167 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/natural_language.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/natural_language.py @@ -22,8 +22,6 @@ from collections.abc import Sequence from typing import TYPE_CHECKING -from airflow.providers.google.common.consts import CLIENT_INFO -from airflow.providers.google.common.hooks.base_google import GoogleBaseHook from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault from google.cloud.language_v1 import EncodingType, LanguageServiceClient from google.cloud.language_v1.types import ( @@ -37,6 +35,9 @@ Document, ) +from airflow.providers.google.common.consts import CLIENT_INFO +from airflow.providers.google.common.hooks.base_google import GoogleBaseHook + if TYPE_CHECKING: from google.api_core.retry import Retry diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/os_login.py b/providers/google/src/airflow/providers/google/cloud/hooks/os_login.py index 0daaa892ae4d9..bcb3d8eac1c72 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/os_login.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/os_login.py @@ -27,11 +27,12 @@ from collections.abc import Sequence from typing import TYPE_CHECKING -from airflow.providers.google.common.consts import CLIENT_INFO -from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID, GoogleBaseHook from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault from google.cloud.oslogin_v1 import ImportSshPublicKeyResponse, OsLoginServiceClient +from airflow.providers.google.common.consts import CLIENT_INFO +from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID, GoogleBaseHook + if TYPE_CHECKING: from google.api_core.retry import Retry diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/pubsub.py b/providers/google/src/airflow/providers/google/cloud/hooks/pubsub.py index dc0b48ac3d73c..58237c5c2ae1e 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/pubsub.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/pubsub.py @@ -33,6 +33,12 @@ from typing import TYPE_CHECKING, Any from uuid import uuid4 +from google.api_core.exceptions import AlreadyExists, GoogleAPICallError +from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault +from google.cloud.exceptions import NotFound +from google.cloud.pubsub_v1 import PublisherClient, SubscriberClient +from google.cloud.pubsub_v1.types import PublisherOptions +from google.pubsub_v1.services.subscriber.async_client import SubscriberAsyncClient from googleapiclient.errors import HttpError from airflow.exceptions import AirflowProviderDeprecationWarning @@ -43,12 +49,6 @@ GoogleBaseHook, ) from airflow.version import version -from google.api_core.exceptions import AlreadyExists, GoogleAPICallError -from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault -from google.cloud.exceptions import NotFound -from google.cloud.pubsub_v1 import PublisherClient, SubscriberClient -from google.cloud.pubsub_v1.types import PublisherOptions -from google.pubsub_v1.services.subscriber.async_client import SubscriberAsyncClient if TYPE_CHECKING: from google.api_core.retry import Retry diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/secret_manager.py b/providers/google/src/airflow/providers/google/cloud/hooks/secret_manager.py index 746c3c22ba446..db144acf42da6 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/secret_manager.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/secret_manager.py @@ -23,8 +23,6 @@ from functools import cached_property from typing import TYPE_CHECKING -from airflow.providers.google.common.consts import CLIENT_INFO -from airflow.providers.google.common.hooks.base_google import GoogleBaseHook from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault from google.cloud.secretmanager_v1 import ( AccessSecretVersionResponse, @@ -34,6 +32,9 @@ SecretVersion, ) +from airflow.providers.google.common.consts import CLIENT_INFO +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 diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/spanner.py b/providers/google/src/airflow/providers/google/cloud/hooks/spanner.py index cdfe3e9350562..24bea1e9566d5 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/spanner.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/spanner.py @@ -22,14 +22,14 @@ from collections.abc import Sequence from typing import TYPE_CHECKING, Callable, NamedTuple +from google.api_core.exceptions import AlreadyExists, GoogleAPICallError +from google.cloud.spanner_v1.client import Client from sqlalchemy import create_engine from airflow.exceptions import AirflowException from airflow.providers.common.sql.hooks.sql import DbApiHook from airflow.providers.google.common.consts import CLIENT_INFO from airflow.providers.google.common.hooks.base_google import GoogleBaseHook, get_field -from google.api_core.exceptions import AlreadyExists, GoogleAPICallError -from google.cloud.spanner_v1.client import Client if TYPE_CHECKING: from google.cloud.spanner_v1.database import Database diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/speech_to_text.py b/providers/google/src/airflow/providers/google/cloud/hooks/speech_to_text.py index 6171079e750d4..098081990f9bb 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/speech_to_text.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/speech_to_text.py @@ -22,12 +22,13 @@ from collections.abc import Sequence from typing import TYPE_CHECKING -from airflow.providers.google.common.consts import CLIENT_INFO -from airflow.providers.google.common.hooks.base_google import GoogleBaseHook from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault from google.cloud.speech_v1 import SpeechClient from google.cloud.speech_v1.types import RecognitionAudio, RecognitionConfig +from airflow.providers.google.common.consts import CLIENT_INFO +from airflow.providers.google.common.hooks.base_google import GoogleBaseHook + if TYPE_CHECKING: from google.api_core.retry import Retry diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/stackdriver.py b/providers/google/src/airflow/providers/google/cloud/hooks/stackdriver.py index c76f2eba6083c..a24c540c73d1c 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/stackdriver.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/stackdriver.py @@ -24,15 +24,15 @@ from collections.abc import Sequence from typing import TYPE_CHECKING, Any -from googleapiclient.errors import HttpError - -from airflow.exceptions import AirflowException -from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID, GoogleBaseHook from google.api_core.exceptions import InvalidArgument from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault from google.cloud import monitoring_v3 from google.cloud.monitoring_v3 import AlertPolicy, NotificationChannel from google.protobuf.field_mask_pb2 import FieldMask +from googleapiclient.errors import HttpError + +from airflow.exceptions import AirflowException +from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID, GoogleBaseHook if TYPE_CHECKING: from google.api_core.retry import Retry diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/tasks.py b/providers/google/src/airflow/providers/google/cloud/hooks/tasks.py index ffaf15f564615..98a859a58e43e 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/tasks.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/tasks.py @@ -22,13 +22,14 @@ from collections.abc import Sequence from typing import TYPE_CHECKING -from airflow.exceptions import AirflowException -from airflow.providers.google.common.consts import CLIENT_INFO -from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID, GoogleBaseHook from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault from google.cloud.tasks_v2 import CloudTasksClient from google.cloud.tasks_v2.types import Queue, Task +from airflow.exceptions import AirflowException +from airflow.providers.google.common.consts import CLIENT_INFO +from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID, GoogleBaseHook + if TYPE_CHECKING: from google.api_core.retry import Retry from google.protobuf.field_mask_pb2 import FieldMask diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/text_to_speech.py b/providers/google/src/airflow/providers/google/cloud/hooks/text_to_speech.py index 715c8736b9403..be2e7a3cd3818 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/text_to_speech.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/text_to_speech.py @@ -22,8 +22,6 @@ from collections.abc import Sequence from typing import TYPE_CHECKING -from airflow.providers.google.common.consts import CLIENT_INFO -from airflow.providers.google.common.hooks.base_google import GoogleBaseHook from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault from google.cloud.texttospeech_v1 import TextToSpeechClient from google.cloud.texttospeech_v1.types import ( @@ -33,6 +31,9 @@ VoiceSelectionParams, ) +from airflow.providers.google.common.consts import CLIENT_INFO +from airflow.providers.google.common.hooks.base_google import GoogleBaseHook + if TYPE_CHECKING: from google.api_core.retry import Retry diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/translate.py b/providers/google/src/airflow/providers/google/cloud/hooks/translate.py index dcdc59db00990..6bb095d9a25e6 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/translate.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/translate.py @@ -25,9 +25,6 @@ cast, ) -from airflow.exceptions import AirflowException -from airflow.providers.google.common.consts import CLIENT_INFO -from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID, GoogleBaseHook from google.api_core.exceptions import GoogleAPICallError from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault from google.api_core.retry import Retry @@ -35,9 +32,11 @@ from google.cloud.translate_v3 import TranslationServiceClient from google.cloud.translate_v3.types.translation_service import GlossaryInputConfig -if TYPE_CHECKING: - from proto import Message +from airflow.exceptions import AirflowException +from airflow.providers.google.common.consts import CLIENT_INFO +from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID, GoogleBaseHook +if TYPE_CHECKING: from google.api_core.operation import Operation from google.cloud.translate_v3.services.translation_service import pagers from google.cloud.translate_v3.types import ( @@ -54,6 +53,7 @@ automl_translation, ) from google.cloud.translate_v3.types.translation_service import Glossary + from proto import Message class WaitOperationNotDoneYetError(Exception): diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/vertex_ai/auto_ml.py b/providers/google/src/airflow/providers/google/cloud/hooks/vertex_ai/auto_ml.py index 6619ed78ca1b3..a61fca8ac68e4 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/vertex_ai/auto_ml.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/vertex_ai/auto_ml.py @@ -23,9 +23,6 @@ from collections.abc import Sequence from typing import TYPE_CHECKING -from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning -from airflow.providers.google.common.deprecated import deprecated -from airflow.providers.google.common.hooks.base_google import GoogleBaseHook from google.api_core.client_options import ClientOptions from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault from google.cloud.aiplatform import ( @@ -39,6 +36,10 @@ ) from google.cloud.aiplatform_v1 import JobServiceClient, PipelineServiceClient +from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning +from airflow.providers.google.common.deprecated import deprecated +from airflow.providers.google.common.hooks.base_google import GoogleBaseHook + if TYPE_CHECKING: from google.api_core.operation import Operation from google.api_core.retry import Retry diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/vertex_ai/batch_prediction_job.py b/providers/google/src/airflow/providers/google/cloud/hooks/vertex_ai/batch_prediction_job.py index 3aea4ada81728..286566a778898 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/vertex_ai/batch_prediction_job.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/vertex_ai/batch_prediction_job.py @@ -23,14 +23,15 @@ from collections.abc import Sequence from typing import TYPE_CHECKING -from airflow.exceptions import AirflowException -from airflow.providers.google.common.consts import CLIENT_INFO -from airflow.providers.google.common.hooks.base_google import GoogleBaseAsyncHook, GoogleBaseHook from google.api_core.client_options import ClientOptions from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault from google.cloud.aiplatform import BatchPredictionJob, Model, explain from google.cloud.aiplatform_v1 import JobServiceAsyncClient, JobServiceClient, JobState, types +from airflow.exceptions import AirflowException +from airflow.providers.google.common.consts import CLIENT_INFO +from airflow.providers.google.common.hooks.base_google import GoogleBaseAsyncHook, GoogleBaseHook + if TYPE_CHECKING: from google.api_core.operation import Operation from google.api_core.retry import AsyncRetry, Retry diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/vertex_ai/custom_job.py b/providers/google/src/airflow/providers/google/cloud/hooks/vertex_ai/custom_job.py index 6ee2ac42cdc9e..9948c7e3acfa3 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/vertex_ai/custom_job.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/vertex_ai/custom_job.py @@ -23,10 +23,6 @@ from collections.abc import Sequence from typing import TYPE_CHECKING, Any -from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning -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 GoogleBaseAsyncHook, GoogleBaseHook from google.api_core.client_options import ClientOptions from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault from google.cloud.aiplatform import ( @@ -46,6 +42,11 @@ types, ) +from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning +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 GoogleBaseAsyncHook, GoogleBaseHook + if TYPE_CHECKING: from google.api_core.operation import Operation from google.api_core.retry import AsyncRetry, Retry diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/vertex_ai/dataset.py b/providers/google/src/airflow/providers/google/cloud/hooks/vertex_ai/dataset.py index 365b396e0e4c6..49461f3f974b0 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/vertex_ai/dataset.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/vertex_ai/dataset.py @@ -22,13 +22,14 @@ from collections.abc import Sequence from typing import TYPE_CHECKING -from airflow.exceptions import AirflowException -from airflow.providers.google.common.consts import CLIENT_INFO -from airflow.providers.google.common.hooks.base_google import GoogleBaseHook from google.api_core.client_options import ClientOptions from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault from google.cloud.aiplatform_v1 import DatasetServiceClient +from airflow.exceptions import AirflowException +from airflow.providers.google.common.consts import CLIENT_INFO +from airflow.providers.google.common.hooks.base_google import GoogleBaseHook + if TYPE_CHECKING: from google.api_core.operation import Operation from google.api_core.retry import Retry diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/vertex_ai/endpoint_service.py b/providers/google/src/airflow/providers/google/cloud/hooks/vertex_ai/endpoint_service.py index b03a2af482d9a..7d8d376ebf38a 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/vertex_ai/endpoint_service.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/vertex_ai/endpoint_service.py @@ -22,12 +22,13 @@ from collections.abc import Sequence from typing import TYPE_CHECKING -from airflow.exceptions import AirflowException -from airflow.providers.google.common.hooks.base_google import GoogleBaseHook from google.api_core.client_options import ClientOptions from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault from google.cloud.aiplatform_v1 import EndpointServiceClient +from airflow.exceptions import AirflowException +from airflow.providers.google.common.hooks.base_google import GoogleBaseHook + if TYPE_CHECKING: from google.api_core.operation import Operation from google.api_core.retry import Retry diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/vertex_ai/feature_store.py b/providers/google/src/airflow/providers/google/cloud/hooks/vertex_ai/feature_store.py index 364e4634928aa..69c7b69f8dad5 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/vertex_ai/feature_store.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/vertex_ai/feature_store.py @@ -18,14 +18,15 @@ from __future__ import annotations -from airflow.exceptions import AirflowException -from airflow.providers.google.common.consts import CLIENT_INFO -from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID, GoogleBaseHook from google.api_core.client_options import ClientOptions from google.cloud.aiplatform_v1beta1 import ( FeatureOnlineStoreAdminServiceClient, ) +from airflow.exceptions import AirflowException +from airflow.providers.google.common.consts import CLIENT_INFO +from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID, GoogleBaseHook + class FeatureStoreHook(GoogleBaseHook): """ diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/vertex_ai/hyperparameter_tuning_job.py b/providers/google/src/airflow/providers/google/cloud/hooks/vertex_ai/hyperparameter_tuning_job.py index 5d145d50a4ba4..a3eec0ef855de 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/vertex_ai/hyperparameter_tuning_job.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/vertex_ai/hyperparameter_tuning_job.py @@ -29,14 +29,15 @@ from collections.abc import Sequence from typing import TYPE_CHECKING -from airflow.exceptions import AirflowException -from airflow.providers.google.common.consts import CLIENT_INFO -from airflow.providers.google.common.hooks.base_google import GoogleBaseAsyncHook, GoogleBaseHook from google.api_core.client_options import ClientOptions from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault from google.cloud.aiplatform import CustomJob, HyperparameterTuningJob, gapic, hyperparameter_tuning from google.cloud.aiplatform_v1 import JobServiceAsyncClient, JobServiceClient, JobState, types +from airflow.exceptions import AirflowException +from airflow.providers.google.common.consts import CLIENT_INFO +from airflow.providers.google.common.hooks.base_google import GoogleBaseAsyncHook, GoogleBaseHook + if TYPE_CHECKING: from google.api_core.operation import Operation from google.api_core.retry import AsyncRetry, Retry diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/vertex_ai/model_service.py b/providers/google/src/airflow/providers/google/cloud/hooks/vertex_ai/model_service.py index 3a192870b1fb2..ed9e8c7723ebd 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/vertex_ai/model_service.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/vertex_ai/model_service.py @@ -23,12 +23,13 @@ from collections.abc import Sequence from typing import TYPE_CHECKING -from airflow.exceptions import AirflowException -from airflow.providers.google.common.hooks.base_google import GoogleBaseHook from google.api_core.client_options import ClientOptions from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault from google.cloud.aiplatform_v1 import ModelServiceClient +from airflow.exceptions import AirflowException +from airflow.providers.google.common.hooks.base_google import GoogleBaseHook + if TYPE_CHECKING: from google.api_core.operation import Operation from google.api_core.retry import Retry diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/vertex_ai/pipeline_job.py b/providers/google/src/airflow/providers/google/cloud/hooks/vertex_ai/pipeline_job.py index 82832825d6953..9d91d08cbd5d8 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/vertex_ai/pipeline_job.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/vertex_ai/pipeline_job.py @@ -29,9 +29,6 @@ from collections.abc import Sequence from typing import TYPE_CHECKING, Any -from airflow.exceptions import AirflowException -from airflow.providers.google.common.consts import CLIENT_INFO -from airflow.providers.google.common.hooks.base_google import GoogleBaseAsyncHook, GoogleBaseHook from google.api_core.client_options import ClientOptions from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault from google.cloud.aiplatform import PipelineJob @@ -42,6 +39,10 @@ types, ) +from airflow.exceptions import AirflowException +from airflow.providers.google.common.consts import CLIENT_INFO +from airflow.providers.google.common.hooks.base_google import GoogleBaseAsyncHook, GoogleBaseHook + if TYPE_CHECKING: from google.api_core.operation import Operation from google.api_core.retry import AsyncRetry, Retry diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/vertex_ai/prediction_service.py b/providers/google/src/airflow/providers/google/cloud/hooks/vertex_ai/prediction_service.py index 9bdc534f81732..9fcc984408892 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/vertex_ai/prediction_service.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/vertex_ai/prediction_service.py @@ -20,12 +20,13 @@ from collections.abc import Sequence from typing import TYPE_CHECKING -from airflow.providers.google.common.consts import CLIENT_INFO -from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID, GoogleBaseHook from google.api_core.client_options import ClientOptions from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault from google.cloud.aiplatform_v1 import PredictionServiceClient +from airflow.providers.google.common.consts import CLIENT_INFO +from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID, GoogleBaseHook + if TYPE_CHECKING: from google.api_core.retry import Retry from google.cloud.aiplatform_v1.types import PredictResponse diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/video_intelligence.py b/providers/google/src/airflow/providers/google/cloud/hooks/video_intelligence.py index e65da3c4bf2f7..df1271b3f75b5 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/video_intelligence.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/video_intelligence.py @@ -22,8 +22,6 @@ from collections.abc import Sequence from typing import TYPE_CHECKING -from airflow.providers.google.common.consts import CLIENT_INFO -from airflow.providers.google.common.hooks.base_google import GoogleBaseHook from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault from google.cloud.videointelligence_v1 import ( Feature, @@ -31,6 +29,9 @@ VideoIntelligenceServiceClient, ) +from airflow.providers.google.common.consts import CLIENT_INFO +from airflow.providers.google.common.hooks.base_google import GoogleBaseHook + if TYPE_CHECKING: from google.api_core.operation import Operation from google.api_core.retry import Retry diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/vision.py b/providers/google/src/airflow/providers/google/cloud/hooks/vision.py index e41020fdd6dd7..a96b75aa13b81 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/vision.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/vision.py @@ -24,9 +24,6 @@ from functools import cached_property from typing import TYPE_CHECKING, Any, Callable -from airflow.exceptions import AirflowException -from airflow.providers.google.common.consts import CLIENT_INFO -from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID, GoogleBaseHook from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault from google.cloud.vision_v1 import ( AnnotateImageRequest, @@ -39,6 +36,10 @@ ) from google.protobuf.json_format import MessageToDict +from airflow.exceptions import AirflowException +from airflow.providers.google.common.consts import CLIENT_INFO +from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID, GoogleBaseHook + if TYPE_CHECKING: from google.api_core.retry import Retry from google.protobuf import field_mask_pb2 diff --git a/providers/google/src/airflow/providers/google/cloud/hooks/workflows.py b/providers/google/src/airflow/providers/google/cloud/hooks/workflows.py index 5bb712c35786d..cd063c270fc93 100644 --- a/providers/google/src/airflow/providers/google/cloud/hooks/workflows.py +++ b/providers/google/src/airflow/providers/google/cloud/hooks/workflows.py @@ -19,12 +19,13 @@ from collections.abc import Sequence from typing import TYPE_CHECKING -from airflow.providers.google.common.consts import CLIENT_INFO -from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID, GoogleBaseHook from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault from google.cloud.workflows.executions_v1beta import Execution, ExecutionsClient from google.cloud.workflows_v1beta import Workflow, WorkflowsClient +from airflow.providers.google.common.consts import CLIENT_INFO +from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID, GoogleBaseHook + if TYPE_CHECKING: from google.api_core.operation import Operation from google.api_core.retry import Retry diff --git a/providers/google/src/airflow/providers/google/cloud/links/kubernetes_engine.py b/providers/google/src/airflow/providers/google/cloud/links/kubernetes_engine.py index 4c010844e3666..4169a48133f78 100644 --- a/providers/google/src/airflow/providers/google/cloud/links/kubernetes_engine.py +++ b/providers/google/src/airflow/providers/google/cloud/links/kubernetes_engine.py @@ -19,9 +19,10 @@ import json from typing import TYPE_CHECKING -from airflow.providers.google.cloud.links.base import BaseGoogleLink from google.cloud.container_v1.types import Cluster +from airflow.providers.google.cloud.links.base import BaseGoogleLink + if TYPE_CHECKING: from airflow.utils.context import Context diff --git a/providers/google/src/airflow/providers/google/cloud/log/gcs_task_handler.py b/providers/google/src/airflow/providers/google/cloud/log/gcs_task_handler.py index 3c4f0f5b4d0ed..ba2cf4db27d97 100644 --- a/providers/google/src/airflow/providers/google/cloud/log/gcs_task_handler.py +++ b/providers/google/src/airflow/providers/google/cloud/log/gcs_task_handler.py @@ -25,6 +25,9 @@ from pathlib import Path from typing import TYPE_CHECKING +# not sure why but mypy complains on missing `storage` but it is clearly there and is importable +from google.cloud import storage # type: ignore[attr-defined] + from airflow.configuration import conf from airflow.exceptions import AirflowNotFoundException from airflow.providers.google.cloud.hooks.gcs import GCSHook, _parse_gcs_url @@ -34,9 +37,6 @@ from airflow.utils.log.file_task_handler import FileTaskHandler from airflow.utils.log.logging_mixin import LoggingMixin -# not sure why but mypy complains on missing `storage` but it is clearly there and is importable -from google.cloud import storage # type: ignore[attr-defined] - if TYPE_CHECKING: from airflow.models.taskinstance import TaskInstance diff --git a/providers/google/src/airflow/providers/google/cloud/log/stackdriver_task_handler.py b/providers/google/src/airflow/providers/google/cloud/log/stackdriver_task_handler.py index 32adf2d92f50e..28d926b082493 100644 --- a/providers/google/src/airflow/providers/google/cloud/log/stackdriver_task_handler.py +++ b/providers/google/src/airflow/providers/google/cloud/log/stackdriver_task_handler.py @@ -25,21 +25,23 @@ from typing import TYPE_CHECKING from urllib.parse import urlencode -from airflow.exceptions import AirflowProviderDeprecationWarning -from airflow.providers.google.cloud.utils.credentials_provider import get_credentials_and_project_id -from airflow.providers.google.common.consts import CLIENT_INFO -from airflow.providers.google.version_compat import AIRFLOW_V_3_0_PLUS -from airflow.utils.types import NOTSET, ArgNotSet from google.cloud import logging as gcp_logging from google.cloud.logging import Resource from google.cloud.logging.handlers.transports import BackgroundThreadTransport, Transport from google.cloud.logging_v2.services.logging_service_v2 import LoggingServiceV2Client from google.cloud.logging_v2.types import ListLogEntriesRequest, ListLogEntriesResponse +from airflow.exceptions import AirflowProviderDeprecationWarning +from airflow.providers.google.cloud.utils.credentials_provider import get_credentials_and_project_id +from airflow.providers.google.common.consts import CLIENT_INFO +from airflow.providers.google.version_compat import AIRFLOW_V_3_0_PLUS +from airflow.utils.types import NOTSET, ArgNotSet + if TYPE_CHECKING: - from airflow.models import TaskInstance from google.auth.credentials import Credentials + from airflow.models import TaskInstance + if not AIRFLOW_V_3_0_PLUS: from airflow.utils.log.trigger_handler import ctx_indiv_trigger diff --git a/providers/google/src/airflow/providers/google/cloud/openlineage/utils.py b/providers/google/src/airflow/providers/google/cloud/openlineage/utils.py index 59f2b2cc8bce0..aabeb581d7aba 100644 --- a/providers/google/src/airflow/providers/google/cloud/openlineage/utils.py +++ b/providers/google/src/airflow/providers/google/cloud/openlineage/utils.py @@ -26,6 +26,8 @@ from collections.abc import Iterable from typing import TYPE_CHECKING, Any +from google.cloud.dataproc_v1 import Batch, RuntimeConfig + from airflow.providers.common.compat.openlineage.facet import ( ColumnLineageDatasetFacet, DatasetFacet, @@ -42,12 +44,12 @@ inject_transport_information_into_spark_properties, ) from airflow.providers.google.cloud.hooks.gcs import _parse_gcs_url -from google.cloud.dataproc_v1 import Batch, RuntimeConfig if TYPE_CHECKING: + from google.cloud.bigquery.table import Table + from airflow.providers.common.compat.openlineage.facet import Dataset from airflow.utils.context import Context - from google.cloud.bigquery.table import Table log = logging.getLogger(__name__) diff --git a/providers/google/src/airflow/providers/google/cloud/operators/alloy_db.py b/providers/google/src/airflow/providers/google/cloud/operators/alloy_db.py index e3ec0fd15356a..634f97b6d9878 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/alloy_db.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/alloy_db.py @@ -23,6 +23,10 @@ from functools import cached_property from typing import TYPE_CHECKING +from google.api_core.exceptions import NotFound +from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault +from google.cloud import alloydb_v1 + from airflow.exceptions import AirflowException from airflow.providers.google.cloud.hooks.alloy_db import AlloyDbHook from airflow.providers.google.cloud.links.alloy_db import ( @@ -31,18 +35,15 @@ AlloyDBUsersLink, ) from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator -from google.api_core.exceptions import NotFound -from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault -from google.cloud import alloydb_v1 if TYPE_CHECKING: import proto - - from airflow.utils.context import Context from google.api_core.operation import Operation from google.api_core.retry import Retry from google.protobuf.field_mask_pb2 import FieldMask + from airflow.utils.context import Context + class AlloyDBBaseOperator(GoogleCloudBaseOperator): """ diff --git a/providers/google/src/airflow/providers/google/cloud/operators/automl.py b/providers/google/src/airflow/providers/google/cloud/operators/automl.py index 8d4a95e1155aa..2a683938ed9ac 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/automl.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/automl.py @@ -24,6 +24,15 @@ from functools import cached_property from typing import TYPE_CHECKING, cast +from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault +from google.cloud.automl_v1beta1 import ( + ColumnSpec, + Dataset, + Model, + PredictResponse, + TableSpec, +) + from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning from airflow.providers.google.cloud.hooks.automl import CloudAutoMLHook from airflow.providers.google.cloud.hooks.vertex_ai.prediction_service import PredictionServiceHook @@ -37,19 +46,12 @@ from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator from airflow.providers.google.common.deprecated import deprecated from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID -from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault -from google.cloud.automl_v1beta1 import ( - ColumnSpec, - Dataset, - Model, - PredictResponse, - TableSpec, -) if TYPE_CHECKING: - from airflow.utils.context import Context from google.api_core.retry import Retry + from airflow.utils.context import Context + MetaData = Sequence[tuple[str, str]] diff --git a/providers/google/src/airflow/providers/google/cloud/operators/bigquery.py b/providers/google/src/airflow/providers/google/cloud/operators/bigquery.py index afc8f7550001d..fdbd990007092 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/bigquery.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/bigquery.py @@ -27,6 +27,11 @@ from functools import cached_property from typing import TYPE_CHECKING, Any, SupportsAbs +from google.api_core.exceptions import Conflict +from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault +from google.cloud.bigquery import DEFAULT_RETRY, CopyJob, ExtractJob, LoadJob, QueryJob, Row +from google.cloud.bigquery.table import RowIterator, Table, TableListItem, TableReference + from airflow.configuration import conf from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning, AirflowSkipException from airflow.providers.common.sql.operators.sql import ( # type: ignore[attr-defined] # for _parse_boolean @@ -57,16 +62,13 @@ 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 -from google.api_core.exceptions import Conflict -from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault -from google.cloud.bigquery import DEFAULT_RETRY, CopyJob, ExtractJob, LoadJob, QueryJob, Row -from google.cloud.bigquery.table import RowIterator, Table, TableListItem, TableReference if TYPE_CHECKING: - from airflow.utils.context import Context from google.api_core.retry import Retry from google.cloud.bigquery import UnknownJob + from airflow.utils.context import Context + BIGQUERY_JOB_DETAILS_LINK_FMT = "https://console.cloud.google.com/bigquery?j={job_id}" diff --git a/providers/google/src/airflow/providers/google/cloud/operators/bigquery_dts.py b/providers/google/src/airflow/providers/google/cloud/operators/bigquery_dts.py index 8a6c751e7529e..bed14add1911f 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/bigquery_dts.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/bigquery_dts.py @@ -24,13 +24,6 @@ from functools import cached_property from typing import TYPE_CHECKING -from airflow.configuration import conf -from airflow.exceptions import AirflowException -from airflow.providers.google.cloud.hooks.bigquery_dts import BiqQueryDataTransferServiceHook, get_object_id -from airflow.providers.google.cloud.links.bigquery_dts import BigQueryDataTransferConfigLink -from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator -from airflow.providers.google.cloud.triggers.bigquery_dts import BigQueryDataTransferRunTrigger -from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault from google.cloud.bigquery_datatransfer_v1 import ( StartManualTransferRunsResponse, @@ -39,10 +32,19 @@ TransferState, ) +from airflow.configuration import conf +from airflow.exceptions import AirflowException +from airflow.providers.google.cloud.hooks.bigquery_dts import BiqQueryDataTransferServiceHook, get_object_id +from airflow.providers.google.cloud.links.bigquery_dts import BigQueryDataTransferConfigLink +from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator +from airflow.providers.google.cloud.triggers.bigquery_dts import BigQueryDataTransferRunTrigger +from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID + if TYPE_CHECKING: - from airflow.utils.context import Context from google.api_core.retry import Retry + from airflow.utils.context import Context + def _get_transfer_config_details(config_transfer_name: str): config_details = config_transfer_name.split("/") diff --git a/providers/google/src/airflow/providers/google/cloud/operators/bigtable.py b/providers/google/src/airflow/providers/google/cloud/operators/bigtable.py index f0c838de2e22f..d33fa159d7b39 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/bigtable.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/bigtable.py @@ -23,6 +23,7 @@ from typing import TYPE_CHECKING import google.api_core.exceptions + from airflow.exceptions import AirflowException from airflow.providers.google.cloud.hooks.bigtable import BigtableHook from airflow.providers.google.cloud.links.bigtable import ( @@ -36,10 +37,11 @@ if TYPE_CHECKING: import enum - from airflow.utils.context import Context from google.cloud.bigtable import enums from google.cloud.bigtable.column_family import GarbageCollectionRule + from airflow.utils.context import Context + class BigtableValidationMixin: """Common class for Cloud Bigtable operators for validating required fields.""" diff --git a/providers/google/src/airflow/providers/google/cloud/operators/cloud_base.py b/providers/google/src/airflow/providers/google/cloud/operators/cloud_base.py index d30c1b90f5600..f5c7af50eba2d 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/cloud_base.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/cloud_base.py @@ -19,9 +19,10 @@ from __future__ import annotations -from airflow.models import BaseOperator from google.api_core.gapic_v1.method import DEFAULT +from airflow.models import BaseOperator + class GoogleCloudBaseOperator(BaseOperator): """Abstract base class for operators using Google API client libraries.""" diff --git a/providers/google/src/airflow/providers/google/cloud/operators/cloud_batch.py b/providers/google/src/airflow/providers/google/cloud/operators/cloud_batch.py index 35a33ca0454ff..0587076f9e263 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/cloud_batch.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/cloud_batch.py @@ -20,17 +20,19 @@ from collections.abc import Sequence from typing import TYPE_CHECKING +from google.cloud.batch_v1 import Job, Task + from airflow.configuration import conf from airflow.exceptions import AirflowException from airflow.providers.google.cloud.hooks.cloud_batch import CloudBatchHook from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator from airflow.providers.google.cloud.triggers.cloud_batch import CloudBatchJobFinishedTrigger -from google.cloud.batch_v1 import Job, Task if TYPE_CHECKING: - from airflow.utils.context import Context from google.api_core import operation + from airflow.utils.context import Context + class CloudBatchSubmitJobOperator(GoogleCloudBaseOperator): """ diff --git a/providers/google/src/airflow/providers/google/cloud/operators/cloud_build.py b/providers/google/src/airflow/providers/google/cloud/operators/cloud_build.py index d326ab5be629e..565b2be13aa27 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/cloud_build.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/cloud_build.py @@ -26,6 +26,9 @@ from typing import TYPE_CHECKING, Any from urllib.parse import unquote, urlsplit +from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault +from google.cloud.devtools.cloudbuild_v1.types import Build, BuildTrigger, RepoSource + from airflow.configuration import conf from airflow.exceptions import AirflowException from airflow.providers.google.cloud.hooks.cloud_build import CloudBuildHook @@ -41,13 +44,12 @@ from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID from airflow.utils import yaml from airflow.utils.helpers import exactly_one -from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault -from google.cloud.devtools.cloudbuild_v1.types import Build, BuildTrigger, RepoSource if TYPE_CHECKING: - from airflow.utils.context import Context from google.api_core.retry import Retry + from airflow.utils.context import Context + REGEX_REPO_PATH = re.compile(r"^/(?P[^/]+)/(?P[^/]+)[\+/]*(?P[^:]+)?") diff --git a/providers/google/src/airflow/providers/google/cloud/operators/cloud_composer.py b/providers/google/src/airflow/providers/google/cloud/operators/cloud_composer.py index dc716d3571b2b..5bc0d11afdbb1 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/cloud_composer.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/cloud_composer.py @@ -21,6 +21,11 @@ from collections.abc import Sequence from typing import TYPE_CHECKING +from google.api_core.exceptions import AlreadyExists +from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault +from google.cloud.orchestration.airflow.service_v1 import ImageVersion +from google.cloud.orchestration.airflow.service_v1.types import Environment, ExecuteAirflowCommandResponse + from airflow.configuration import conf from airflow.exceptions import AirflowException from airflow.providers.google.cloud.hooks.cloud_composer import CloudComposerHook @@ -31,16 +36,13 @@ CloudComposerExecutionTrigger, ) from airflow.providers.google.common.consts import GOOGLE_DEFAULT_DEFERRABLE_METHOD_NAME -from google.api_core.exceptions import AlreadyExists -from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault -from google.cloud.orchestration.airflow.service_v1 import ImageVersion -from google.cloud.orchestration.airflow.service_v1.types import Environment, ExecuteAirflowCommandResponse if TYPE_CHECKING: - from airflow.utils.context import Context from google.api_core.retry import Retry from google.protobuf.field_mask_pb2 import FieldMask + from airflow.utils.context import Context + CLOUD_COMPOSER_BASE_LINK = "https://console.cloud.google.com/composer/environments" CLOUD_COMPOSER_DETAILS_LINK = ( CLOUD_COMPOSER_BASE_LINK + "/detail/{region}/{environment_id}/monitoring?project={project_id}" diff --git a/providers/google/src/airflow/providers/google/cloud/operators/cloud_memorystore.py b/providers/google/src/airflow/providers/google/cloud/operators/cloud_memorystore.py index 5e80a8786a211..5138ed1cf5143 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/cloud_memorystore.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/cloud_memorystore.py @@ -29,6 +29,10 @@ from collections.abc import Sequence from typing import TYPE_CHECKING +from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault +from google.cloud.memcache_v1beta2.types import cloud_memcache +from google.cloud.redis_v1 import FailoverInstanceRequest, InputConfig, Instance, OutputConfig + from airflow.providers.google.cloud.hooks.cloud_memorystore import ( CloudMemorystoreHook, CloudMemorystoreMemcachedHook, @@ -41,15 +45,13 @@ ) from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID -from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault -from google.cloud.memcache_v1beta2.types import cloud_memcache -from google.cloud.redis_v1 import FailoverInstanceRequest, InputConfig, Instance, OutputConfig if TYPE_CHECKING: - from airflow.utils.context import Context from google.api_core.retry import Retry from google.protobuf.field_mask_pb2 import FieldMask + from airflow.utils.context import Context + class CloudMemorystoreCreateInstanceOperator(GoogleCloudBaseOperator): """ diff --git a/providers/google/src/airflow/providers/google/cloud/operators/cloud_run.py b/providers/google/src/airflow/providers/google/cloud/operators/cloud_run.py index 571356cc2afb8..1b5e433b7d017 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/cloud_run.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/cloud_run.py @@ -21,19 +21,21 @@ from typing import TYPE_CHECKING, Any import google.cloud.exceptions +from google.api_core.exceptions import AlreadyExists +from google.cloud.run_v2 import Job, Service + from airflow.configuration import conf from airflow.exceptions import AirflowException from airflow.providers.google.cloud.hooks.cloud_run import CloudRunHook, CloudRunServiceHook from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator from airflow.providers.google.cloud.triggers.cloud_run import CloudRunJobFinishedTrigger, RunJobStatus -from google.api_core.exceptions import AlreadyExists -from google.cloud.run_v2 import Job, Service if TYPE_CHECKING: - from airflow.utils.context import Context from google.api_core import operation from google.cloud.run_v2.types import Execution + from airflow.utils.context import Context + class CloudRunCreateJobOperator(GoogleCloudBaseOperator): """ diff --git a/providers/google/src/airflow/providers/google/cloud/operators/compute.py b/providers/google/src/airflow/providers/google/cloud/operators/compute.py index 9b85e7f799f4f..b43305c13af33 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/compute.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/compute.py @@ -23,6 +23,8 @@ from copy import deepcopy from typing import TYPE_CHECKING, Any +from google.api_core import exceptions +from google.cloud.compute_v1.types import Instance, InstanceGroupManager, InstanceTemplate from json_merge_patch import merge from airflow.exceptions import AirflowException @@ -36,13 +38,12 @@ from airflow.providers.google.cloud.utils.field_sanitizer import GcpBodyFieldSanitizer from airflow.providers.google.cloud.utils.field_validator import GcpBodyFieldValidator from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID -from google.api_core import exceptions -from google.cloud.compute_v1.types import Instance, InstanceGroupManager, InstanceTemplate if TYPE_CHECKING: - from airflow.utils.context import Context from google.api_core.retry import Retry + from airflow.utils.context import Context + class ComputeEngineBaseOperator(GoogleCloudBaseOperator): """Abstract base operator for Google Compute Engine operators to inherit from.""" diff --git a/providers/google/src/airflow/providers/google/cloud/operators/datacatalog.py b/providers/google/src/airflow/providers/google/cloud/operators/datacatalog.py index 757581ebe0d25..13d29c5772fd4 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/datacatalog.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/datacatalog.py @@ -19,14 +19,6 @@ from collections.abc import Sequence from typing import TYPE_CHECKING -from airflow.providers.google.cloud.hooks.datacatalog import CloudDataCatalogHook -from airflow.providers.google.cloud.links.datacatalog import ( - DataCatalogEntryGroupLink, - DataCatalogEntryLink, - DataCatalogTagTemplateLink, -) -from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator -from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID from google.api_core.exceptions import AlreadyExists, NotFound from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault from google.cloud.datacatalog import ( @@ -40,11 +32,21 @@ TagTemplateField, ) +from airflow.providers.google.cloud.hooks.datacatalog import CloudDataCatalogHook +from airflow.providers.google.cloud.links.datacatalog import ( + DataCatalogEntryGroupLink, + DataCatalogEntryLink, + DataCatalogTagTemplateLink, +) +from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator +from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID + if TYPE_CHECKING: - from airflow.utils.context import Context from google.api_core.retry import Retry from google.protobuf.field_mask_pb2 import FieldMask + from airflow.utils.context import Context + class CloudDataCatalogCreateEntryOperator(GoogleCloudBaseOperator): """ diff --git a/providers/google/src/airflow/providers/google/cloud/operators/dataform.py b/providers/google/src/airflow/providers/google/cloud/operators/dataform.py index a56d47eb966bc..4212cd22a12ec 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/dataform.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/dataform.py @@ -26,11 +26,10 @@ ) if TYPE_CHECKING: - from airflow.utils.context import Context from google.api_core.retry import Retry -from airflow.providers.google.cloud.hooks.dataform import DataformHook -from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator + from airflow.utils.context import Context + from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault from google.cloud.dataform_v1beta1.types import ( CompilationResult, @@ -43,6 +42,9 @@ WriteFileResponse, ) +from airflow.providers.google.cloud.hooks.dataform import DataformHook +from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator + class DataformCreateCompilationResultOperator(GoogleCloudBaseOperator): """ diff --git a/providers/google/src/airflow/providers/google/cloud/operators/datafusion.py b/providers/google/src/airflow/providers/google/cloud/operators/datafusion.py index d33139b323005..02169748b3984 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/datafusion.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/datafusion.py @@ -22,6 +22,7 @@ from collections.abc import Sequence from typing import TYPE_CHECKING, Any +from google.api_core.retry import exponential_sleep_generator from googleapiclient.errors import HttpError from airflow.configuration import conf @@ -38,7 +39,6 @@ from airflow.providers.google.cloud.utils.helpers import resource_path_to_dict from airflow.providers.google.common.deprecated import deprecated from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID -from google.api_core.retry import exponential_sleep_generator if TYPE_CHECKING: from airflow.utils.context import Context diff --git a/providers/google/src/airflow/providers/google/cloud/operators/dataplex.py b/providers/google/src/airflow/providers/google/cloud/operators/dataplex.py index 65ef0e0dd18dd..2c2f7833c9762 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/dataplex.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/dataplex.py @@ -23,34 +23,19 @@ from functools import cached_property from typing import TYPE_CHECKING, Any +from google.protobuf.json_format import MessageToDict + from airflow.exceptions import AirflowException from airflow.providers.google.cloud.triggers.dataplex import ( DataplexDataProfileJobTrigger, DataplexDataQualityJobTrigger, ) -from google.protobuf.json_format import MessageToDict if TYPE_CHECKING: - from airflow.utils.context import Context from google.protobuf.field_mask_pb2 import FieldMask -from googleapiclient.errors import HttpError + from airflow.utils.context import Context -from airflow.configuration import conf -from airflow.providers.google.cloud.hooks.dataplex import AirflowDataQualityScanException, DataplexHook -from airflow.providers.google.cloud.links.dataplex import ( - DataplexCatalogAspectTypeLink, - DataplexCatalogAspectTypesLink, - DataplexCatalogEntryGroupLink, - DataplexCatalogEntryGroupsLink, - DataplexCatalogEntryLink, - DataplexCatalogEntryTypeLink, - DataplexCatalogEntryTypesLink, - DataplexLakeLink, - DataplexTaskLink, - DataplexTasksLink, -) -from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator from google.api_core.exceptions import AlreadyExists, GoogleAPICallError, NotFound from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault from google.api_core.retry import Retry, exponential_sleep_generator @@ -72,6 +57,23 @@ Task, Zone, ) +from googleapiclient.errors import HttpError + +from airflow.configuration import conf +from airflow.providers.google.cloud.hooks.dataplex import AirflowDataQualityScanException, DataplexHook +from airflow.providers.google.cloud.links.dataplex import ( + DataplexCatalogAspectTypeLink, + DataplexCatalogAspectTypesLink, + DataplexCatalogEntryGroupLink, + DataplexCatalogEntryGroupsLink, + DataplexCatalogEntryLink, + DataplexCatalogEntryTypeLink, + DataplexCatalogEntryTypesLink, + DataplexLakeLink, + DataplexTaskLink, + DataplexTasksLink, +) +from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator class DataplexCreateTaskOperator(GoogleCloudBaseOperator): diff --git a/providers/google/src/airflow/providers/google/cloud/operators/dataproc.py b/providers/google/src/airflow/providers/google/cloud/operators/dataproc.py index 3ceb773fd64f4..eff64faff87f9 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/dataproc.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/dataproc.py @@ -30,6 +30,11 @@ from functools import cached_property from typing import TYPE_CHECKING, Any +from google.api_core.exceptions import AlreadyExists, NotFound +from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault +from google.api_core.retry import Retry, exponential_sleep_generator +from google.cloud.dataproc_v1 import Batch, Cluster, ClusterStatus, JobStatus + from airflow.configuration import conf from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning from airflow.providers.google.cloud.hooks.dataproc import ( @@ -61,19 +66,16 @@ from airflow.providers.google.common.deprecated import deprecated from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID from airflow.utils import timezone -from google.api_core.exceptions import AlreadyExists, NotFound -from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault -from google.api_core.retry import Retry, exponential_sleep_generator -from google.cloud.dataproc_v1 import Batch, Cluster, ClusterStatus, JobStatus if TYPE_CHECKING: - from airflow.utils.context import Context from google.api_core import operation from google.api_core.retry_async import AsyncRetry from google.protobuf.duration_pb2 import Duration from google.protobuf.field_mask_pb2 import FieldMask from google.type.interval_pb2 import Interval + from airflow.utils.context import Context + class PreemptibilityType(Enum): """Contains possible Type values of Preemptibility applicable for every secondary worker of Cluster.""" diff --git a/providers/google/src/airflow/providers/google/cloud/operators/dataproc_metastore.py b/providers/google/src/airflow/providers/google/cloud/operators/dataproc_metastore.py index 410b80fbbeeca..a9470b17e3e3c 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/dataproc_metastore.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/dataproc_metastore.py @@ -23,12 +23,6 @@ from collections.abc import Sequence from typing import TYPE_CHECKING -from airflow.exceptions import AirflowException -from airflow.models import BaseOperator, BaseOperatorLink -from airflow.models.xcom import XCom -from airflow.providers.google.cloud.hooks.dataproc_metastore import DataprocMetastoreHook -from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator -from airflow.providers.google.common.links.storage import StorageLink from google.api_core.exceptions import AlreadyExists from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault from google.api_core.retry import Retry, exponential_sleep_generator @@ -36,10 +30,18 @@ from google.cloud.metastore_v1.types import Backup, MetadataImport, Service from google.cloud.metastore_v1.types.metastore import DatabaseDumpSpec, Restore +from airflow.exceptions import AirflowException +from airflow.models import BaseOperator, BaseOperatorLink +from airflow.models.xcom import XCom +from airflow.providers.google.cloud.hooks.dataproc_metastore import DataprocMetastoreHook +from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator +from airflow.providers.google.common.links.storage import StorageLink + if TYPE_CHECKING: + from google.protobuf.field_mask_pb2 import FieldMask + from airflow.models.taskinstancekey import TaskInstanceKey from airflow.utils.context import Context - from google.protobuf.field_mask_pb2 import FieldMask BASE_LINK = "https://console.cloud.google.com" diff --git a/providers/google/src/airflow/providers/google/cloud/operators/dlp.py b/providers/google/src/airflow/providers/google/cloud/operators/dlp.py index 27809f9b4cc5d..eca84824f0543 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/dlp.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/dlp.py @@ -22,22 +22,6 @@ from collections.abc import Sequence from typing import TYPE_CHECKING -from airflow.providers.google.cloud.hooks.dlp import CloudDLPHook -from airflow.providers.google.cloud.links.data_loss_prevention import ( - CloudDLPDeidentifyTemplateDetailsLink, - CloudDLPDeidentifyTemplatesListLink, - CloudDLPInfoTypeDetailsLink, - CloudDLPInfoTypesListLink, - CloudDLPInspectTemplateDetailsLink, - CloudDLPInspectTemplatesListLink, - CloudDLPJobDetailsLink, - CloudDLPJobsListLink, - CloudDLPJobTriggerDetailsLink, - CloudDLPJobTriggersListLink, - CloudDLPPossibleInfoTypesListLink, -) -from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator -from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID from google.api_core.exceptions import AlreadyExists, InvalidArgument, NotFound from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault from google.cloud.dlp_v2.types import ( @@ -61,11 +45,29 @@ StoredInfoTypeConfig, ) +from airflow.providers.google.cloud.hooks.dlp import CloudDLPHook +from airflow.providers.google.cloud.links.data_loss_prevention import ( + CloudDLPDeidentifyTemplateDetailsLink, + CloudDLPDeidentifyTemplatesListLink, + CloudDLPInfoTypeDetailsLink, + CloudDLPInfoTypesListLink, + CloudDLPInspectTemplateDetailsLink, + CloudDLPInspectTemplatesListLink, + CloudDLPJobDetailsLink, + CloudDLPJobsListLink, + CloudDLPJobTriggerDetailsLink, + CloudDLPJobTriggersListLink, + CloudDLPPossibleInfoTypesListLink, +) +from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator +from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID + if TYPE_CHECKING: - from airflow.utils.context import Context from google.api_core.retry import Retry from google.protobuf.field_mask_pb2 import FieldMask + from airflow.utils.context import Context + class CloudDLPCancelDLPJobOperator(GoogleCloudBaseOperator): """ diff --git a/providers/google/src/airflow/providers/google/cloud/operators/gcs.py b/providers/google/src/airflow/providers/google/cloud/operators/gcs.py index ab529decdcaa0..8fc8eadeaedc6 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/gcs.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/gcs.py @@ -35,13 +35,14 @@ if TYPE_CHECKING: from airflow.utils.context import Context +from google.api_core.exceptions import Conflict +from google.cloud.exceptions import GoogleCloudError + from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning from airflow.providers.google.cloud.hooks.gcs import GCSHook from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator from airflow.providers.google.common.links.storage import FileDetailsLink, StorageLink from airflow.utils import timezone -from google.api_core.exceptions import Conflict -from google.cloud.exceptions import GoogleCloudError class GCSCreateBucketOperator(GoogleCloudBaseOperator): diff --git a/providers/google/src/airflow/providers/google/cloud/operators/kubernetes_engine.py b/providers/google/src/airflow/providers/google/cloud/operators/kubernetes_engine.py index 198c488703755..5288c3b4f9f08 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/kubernetes_engine.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/kubernetes_engine.py @@ -24,6 +24,7 @@ from functools import cached_property from typing import TYPE_CHECKING, Any +from google.api_core.exceptions import AlreadyExists from kubernetes.client import V1JobList, models as k8s from packaging.version import parse as parse_version @@ -60,7 +61,6 @@ from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID from airflow.providers_manager import ProvidersManager from airflow.utils.timezone import utcnow -from google.api_core.exceptions import AlreadyExists try: from airflow.providers.cncf.kubernetes.operators.job import KubernetesDeleteJobOperator @@ -73,11 +73,11 @@ ) if TYPE_CHECKING: + from google.cloud.container_v1.types import Cluster from kubernetes.client.models import V1Job from pendulum import DateTime from airflow.utils.context import Context - from google.cloud.container_v1.types import Cluster KUBE_CONFIG_ENV_VAR = "KUBECONFIG" diff --git a/providers/google/src/airflow/providers/google/cloud/operators/managed_kafka.py b/providers/google/src/airflow/providers/google/cloud/operators/managed_kafka.py index 2afb30fede904..0ded649858f62 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/managed_kafka.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/managed_kafka.py @@ -23,6 +23,10 @@ from functools import cached_property from typing import TYPE_CHECKING +from google.api_core.exceptions import AlreadyExists, NotFound +from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault +from google.cloud.managedkafka_v1 import types + from airflow.exceptions import AirflowException from airflow.providers.google.cloud.hooks.managed_kafka import ManagedKafkaHook from airflow.providers.google.cloud.links.managed_kafka import ( @@ -31,15 +35,13 @@ ApacheKafkaTopicLink, ) from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator -from google.api_core.exceptions import AlreadyExists, NotFound -from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault -from google.cloud.managedkafka_v1 import types if TYPE_CHECKING: - from airflow.utils.context import Context from google.api_core.retry import Retry from google.protobuf.field_mask_pb2 import FieldMask + from airflow.utils.context import Context + class ManagedKafkaBaseOperator(GoogleCloudBaseOperator): """ diff --git a/providers/google/src/airflow/providers/google/cloud/operators/natural_language.py b/providers/google/src/airflow/providers/google/cloud/operators/natural_language.py index 1ac83001d8f59..edd9a4c2ac820 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/natural_language.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/natural_language.py @@ -22,16 +22,18 @@ from collections.abc import Sequence from typing import TYPE_CHECKING -from airflow.providers.google.cloud.hooks.natural_language import CloudNaturalLanguageHook -from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault from google.protobuf.json_format import MessageToDict +from airflow.providers.google.cloud.hooks.natural_language import CloudNaturalLanguageHook +from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator + if TYPE_CHECKING: - from airflow.utils.context import Context from google.api_core.retry import Retry from google.cloud.language_v1.types import Document, EncodingType + from airflow.utils.context import Context + MetaData = Sequence[tuple[str, str]] diff --git a/providers/google/src/airflow/providers/google/cloud/operators/pubsub.py b/providers/google/src/airflow/providers/google/cloud/operators/pubsub.py index 96cf63546fdbc..c7ca9e3e64951 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/pubsub.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/pubsub.py @@ -28,13 +28,6 @@ from collections.abc import Sequence from typing import TYPE_CHECKING, Any, Callable -from airflow.exceptions import AirflowException -from airflow.providers.google.cloud.hooks.pubsub import PubSubHook -from airflow.providers.google.cloud.links.pubsub import PubSubSubscriptionLink, PubSubTopicLink -from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator -from airflow.providers.google.cloud.triggers.pubsub import PubsubPullTrigger -from airflow.providers.google.common.consts import GOOGLE_DEFAULT_DEFERRABLE_METHOD_NAME -from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault from google.cloud.pubsub_v1.types import ( DeadLetterPolicy, @@ -47,10 +40,19 @@ SchemaSettings, ) +from airflow.exceptions import AirflowException +from airflow.providers.google.cloud.hooks.pubsub import PubSubHook +from airflow.providers.google.cloud.links.pubsub import PubSubSubscriptionLink, PubSubTopicLink +from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator +from airflow.providers.google.cloud.triggers.pubsub import PubsubPullTrigger +from airflow.providers.google.common.consts import GOOGLE_DEFAULT_DEFERRABLE_METHOD_NAME +from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID + if TYPE_CHECKING: - from airflow.utils.context import Context from google.api_core.retry import Retry + from airflow.utils.context import Context + class PubSubCreateTopicOperator(GoogleCloudBaseOperator): """ diff --git a/providers/google/src/airflow/providers/google/cloud/operators/speech_to_text.py b/providers/google/src/airflow/providers/google/cloud/operators/speech_to_text.py index 80d372346ddd9..752e7d3454be0 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/speech_to_text.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/speech_to_text.py @@ -22,19 +22,21 @@ from collections.abc import Sequence from typing import TYPE_CHECKING +from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault +from google.protobuf.json_format import MessageToDict + from airflow.exceptions import AirflowException from airflow.providers.google.cloud.hooks.speech_to_text import CloudSpeechToTextHook, RecognitionAudio from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID from airflow.providers.google.common.links.storage import FileDetailsLink -from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault -from google.protobuf.json_format import MessageToDict if TYPE_CHECKING: - from airflow.utils.context import Context from google.api_core.retry import Retry from google.cloud.speech_v1.types import RecognitionConfig + from airflow.utils.context import Context + class CloudSpeechToTextRecognizeSpeechOperator(GoogleCloudBaseOperator): """ diff --git a/providers/google/src/airflow/providers/google/cloud/operators/stackdriver.py b/providers/google/src/airflow/providers/google/cloud/operators/stackdriver.py index 198de1d968597..2edc126479e12 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/stackdriver.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/stackdriver.py @@ -20,6 +20,9 @@ from collections.abc import Sequence from typing import TYPE_CHECKING +from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault +from google.cloud.monitoring_v3 import AlertPolicy, NotificationChannel + from airflow.providers.google.cloud.hooks.stackdriver import StackdriverHook from airflow.providers.google.cloud.links.stackdriver import ( StackdriverNotificationsLink, @@ -27,13 +30,12 @@ ) from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID -from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault -from google.cloud.monitoring_v3 import AlertPolicy, NotificationChannel if TYPE_CHECKING: - from airflow.utils.context import Context from google.api_core.retry import Retry + from airflow.utils.context import Context + class StackdriverListAlertPoliciesOperator(GoogleCloudBaseOperator): """ diff --git a/providers/google/src/airflow/providers/google/cloud/operators/tasks.py b/providers/google/src/airflow/providers/google/cloud/operators/tasks.py index 2125214b7cc3a..735f8fbde9fa9 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/tasks.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/tasks.py @@ -22,19 +22,21 @@ from collections.abc import Sequence from typing import TYPE_CHECKING +from google.api_core.exceptions import AlreadyExists +from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault +from google.cloud.tasks_v2.types import Queue, Task + from airflow.providers.google.cloud.hooks.tasks import CloudTasksHook from airflow.providers.google.cloud.links.cloud_tasks import CloudTasksLink, CloudTasksQueueLink from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID -from google.api_core.exceptions import AlreadyExists -from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault -from google.cloud.tasks_v2.types import Queue, Task if TYPE_CHECKING: - from airflow.utils.context import Context from google.api_core.retry import Retry from google.protobuf.field_mask_pb2 import FieldMask + from airflow.utils.context import Context + MetaData = Sequence[tuple[str, str]] diff --git a/providers/google/src/airflow/providers/google/cloud/operators/text_to_speech.py b/providers/google/src/airflow/providers/google/cloud/operators/text_to_speech.py index fd0d779aa95d3..adb4bb6a9d376 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/text_to_speech.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/text_to_speech.py @@ -23,19 +23,21 @@ from tempfile import NamedTemporaryFile from typing import TYPE_CHECKING +from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault + from airflow.exceptions import AirflowException from airflow.providers.google.cloud.hooks.gcs import GCSHook from airflow.providers.google.cloud.hooks.text_to_speech import CloudTextToSpeechHook from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID from airflow.providers.google.common.links.storage import FileDetailsLink -from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault if TYPE_CHECKING: - from airflow.utils.context import Context from google.api_core.retry import Retry from google.cloud.texttospeech_v1.types import AudioConfig, SynthesisInput, VoiceSelectionParams + from airflow.utils.context import Context + class CloudTextToSpeechSynthesizeOperator(GoogleCloudBaseOperator): """ diff --git a/providers/google/src/airflow/providers/google/cloud/operators/translate.py b/providers/google/src/airflow/providers/google/cloud/operators/translate.py index 265a35f36a1d5..2c2424ea5f41e 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/translate.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/translate.py @@ -22,6 +22,9 @@ from collections.abc import MutableMapping, MutableSequence, Sequence from typing import TYPE_CHECKING, cast +from google.api_core.exceptions import GoogleAPICallError +from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault + from airflow.exceptions import AirflowException from airflow.providers.google.cloud.hooks.translate import CloudTranslateHook, TranslateHook from airflow.providers.google.cloud.links.translate import ( @@ -35,11 +38,8 @@ ) from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID -from google.api_core.exceptions import GoogleAPICallError -from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault if TYPE_CHECKING: - from airflow.utils.context import Context from google.api_core.retry import Retry from google.cloud.translate_v3.types import ( BatchDocumentInputConfig, @@ -55,6 +55,8 @@ ) from google.cloud.translate_v3.types.translation_service import Glossary, GlossaryInputConfig + from airflow.utils.context import Context + class CloudTranslateTextOperator(GoogleCloudBaseOperator): """ diff --git a/providers/google/src/airflow/providers/google/cloud/operators/translate_speech.py b/providers/google/src/airflow/providers/google/cloud/operators/translate_speech.py index f7d85fcf60b9d..e92865beda17f 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/translate_speech.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/translate_speech.py @@ -22,18 +22,20 @@ from collections.abc import Sequence from typing import TYPE_CHECKING +from google.protobuf.json_format import MessageToDict + from airflow.exceptions import AirflowException from airflow.providers.google.cloud.hooks.speech_to_text import CloudSpeechToTextHook from airflow.providers.google.cloud.hooks.translate import CloudTranslateHook from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID from airflow.providers.google.common.links.storage import FileDetailsLink -from google.protobuf.json_format import MessageToDict if TYPE_CHECKING: - from airflow.utils.context import Context from google.cloud.speech_v1.types import RecognitionAudio, RecognitionConfig + from airflow.utils.context import Context + class CloudTranslateSpeechOperator(GoogleCloudBaseOperator): """ diff --git a/providers/google/src/airflow/providers/google/cloud/operators/vertex_ai/auto_ml.py b/providers/google/src/airflow/providers/google/cloud/operators/vertex_ai/auto_ml.py index 5c112aeecdf80..d92b2cef5858d 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/vertex_ai/auto_ml.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/vertex_ai/auto_ml.py @@ -23,6 +23,12 @@ from collections.abc import Sequence from typing import TYPE_CHECKING +from google.api_core.exceptions import NotFound +from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault +from google.cloud.aiplatform import datasets +from google.cloud.aiplatform.models import Model +from google.cloud.aiplatform_v1.types.training_pipeline import TrainingPipeline + from airflow.exceptions import AirflowProviderDeprecationWarning from airflow.providers.google.cloud.hooks.vertex_ai.auto_ml import AutoMLHook from airflow.providers.google.cloud.links.vertex_ai import ( @@ -32,16 +38,12 @@ ) from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator from airflow.providers.google.common.deprecated import deprecated -from google.api_core.exceptions import NotFound -from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault -from google.cloud.aiplatform import datasets -from google.cloud.aiplatform.models import Model -from google.cloud.aiplatform_v1.types.training_pipeline import TrainingPipeline if TYPE_CHECKING: - from airflow.utils.context import Context from google.api_core.retry import Retry + from airflow.utils.context import Context + class AutoMLTrainingJobBaseOperator(GoogleCloudBaseOperator): """The base class for operators that launch AutoML jobs on VertexAI.""" diff --git a/providers/google/src/airflow/providers/google/cloud/operators/vertex_ai/batch_prediction_job.py b/providers/google/src/airflow/providers/google/cloud/operators/vertex_ai/batch_prediction_job.py index 8850ce8a15f11..705f6e31154fd 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/vertex_ai/batch_prediction_job.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/vertex_ai/batch_prediction_job.py @@ -24,6 +24,10 @@ from functools import cached_property from typing import TYPE_CHECKING, Any +from google.api_core.exceptions import NotFound +from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault +from google.cloud.aiplatform_v1.types import BatchPredictionJob + from airflow.configuration import conf from airflow.exceptions import AirflowException from airflow.providers.google.cloud.hooks.vertex_ai.batch_prediction_job import BatchPredictionJobHook @@ -33,15 +37,13 @@ ) from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator from airflow.providers.google.cloud.triggers.vertex_ai import CreateBatchPredictionJobTrigger -from google.api_core.exceptions import NotFound -from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault -from google.cloud.aiplatform_v1.types import BatchPredictionJob if TYPE_CHECKING: - from airflow.utils.context import Context from google.api_core.retry import Retry from google.cloud.aiplatform import BatchPredictionJob as BatchPredictionJobObject, Model, explain + from airflow.utils.context import Context + class CreateBatchPredictionJobOperator(GoogleCloudBaseOperator): """ diff --git a/providers/google/src/airflow/providers/google/cloud/operators/vertex_ai/custom_job.py b/providers/google/src/airflow/providers/google/cloud/operators/vertex_ai/custom_job.py index f3c5be6578327..fa901b4cae654 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/vertex_ai/custom_job.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/vertex_ai/custom_job.py @@ -23,6 +23,12 @@ from functools import cached_property from typing import TYPE_CHECKING, Any +from google.api_core.exceptions import NotFound +from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault +from google.cloud.aiplatform.models import Model +from google.cloud.aiplatform_v1.types.dataset import Dataset +from google.cloud.aiplatform_v1.types.training_pipeline import TrainingPipeline + from airflow.configuration import conf from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning from airflow.providers.google.cloud.hooks.vertex_ai.custom_job import CustomJobHook @@ -38,14 +44,8 @@ CustomTrainingJobTrigger, ) from airflow.providers.google.common.deprecated import deprecated -from google.api_core.exceptions import NotFound -from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault -from google.cloud.aiplatform.models import Model -from google.cloud.aiplatform_v1.types.dataset import Dataset -from google.cloud.aiplatform_v1.types.training_pipeline import TrainingPipeline if TYPE_CHECKING: - from airflow.utils.context import Context from google.api_core.retry import Retry from google.cloud.aiplatform import ( CustomContainerTrainingJob, @@ -53,6 +53,8 @@ CustomTrainingJob, ) + from airflow.utils.context import Context + class CustomTrainingJobBaseOperator(GoogleCloudBaseOperator): """The base class for operators that launch Custom jobs on VertexAI.""" diff --git a/providers/google/src/airflow/providers/google/cloud/operators/vertex_ai/dataset.py b/providers/google/src/airflow/providers/google/cloud/operators/vertex_ai/dataset.py index 53c224eb96cac..0d5754fc3f9e0 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/vertex_ai/dataset.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/vertex_ai/dataset.py @@ -22,18 +22,20 @@ from collections.abc import Sequence from typing import TYPE_CHECKING -from airflow.providers.google.cloud.hooks.vertex_ai.dataset import DatasetHook -from airflow.providers.google.cloud.links.vertex_ai import VertexAIDatasetLink, VertexAIDatasetListLink -from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator from google.api_core.exceptions import NotFound from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault from google.cloud.aiplatform_v1.types import Dataset, ExportDataConfig, ImportDataConfig +from airflow.providers.google.cloud.hooks.vertex_ai.dataset import DatasetHook +from airflow.providers.google.cloud.links.vertex_ai import VertexAIDatasetLink, VertexAIDatasetListLink +from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator + if TYPE_CHECKING: - from airflow.utils.context import Context from google.api_core.retry import Retry from google.protobuf.field_mask_pb2 import FieldMask + from airflow.utils.context import Context + class CreateDatasetOperator(GoogleCloudBaseOperator): """ diff --git a/providers/google/src/airflow/providers/google/cloud/operators/vertex_ai/endpoint_service.py b/providers/google/src/airflow/providers/google/cloud/operators/vertex_ai/endpoint_service.py index 2f58beb1839ec..db485d5cfcc77 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/vertex_ai/endpoint_service.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/vertex_ai/endpoint_service.py @@ -23,6 +23,10 @@ from collections.abc import Sequence from typing import TYPE_CHECKING +from google.api_core.exceptions import NotFound +from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault +from google.cloud.aiplatform_v1.types import DeployedModel, Endpoint, endpoint_service + from airflow.providers.google.cloud.hooks.vertex_ai.endpoint_service import EndpointServiceHook from airflow.providers.google.cloud.links.vertex_ai import ( VertexAIEndpointLink, @@ -30,15 +34,13 @@ VertexAIModelLink, ) from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator -from google.api_core.exceptions import NotFound -from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault -from google.cloud.aiplatform_v1.types import DeployedModel, Endpoint, endpoint_service if TYPE_CHECKING: - from airflow.utils.context import Context from google.api_core.retry import Retry from google.protobuf.field_mask_pb2 import FieldMask + from airflow.utils.context import Context + class CreateEndpointOperator(GoogleCloudBaseOperator): """ diff --git a/providers/google/src/airflow/providers/google/cloud/operators/vertex_ai/hyperparameter_tuning_job.py b/providers/google/src/airflow/providers/google/cloud/operators/vertex_ai/hyperparameter_tuning_job.py index 4ef92c33dfe5e..b0928f67ca9b0 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/vertex_ai/hyperparameter_tuning_job.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/vertex_ai/hyperparameter_tuning_job.py @@ -23,6 +23,10 @@ from collections.abc import Sequence from typing import TYPE_CHECKING, Any +from google.api_core.exceptions import NotFound +from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault +from google.cloud.aiplatform_v1 import types + from airflow.configuration import conf from airflow.exceptions import AirflowException from airflow.providers.google.cloud.hooks.vertex_ai.hyperparameter_tuning_job import ( @@ -34,15 +38,13 @@ ) from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator from airflow.providers.google.cloud.triggers.vertex_ai import CreateHyperparameterTuningJobTrigger -from google.api_core.exceptions import NotFound -from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault -from google.cloud.aiplatform_v1 import types if TYPE_CHECKING: - from airflow.utils.context import Context from google.api_core.retry import Retry from google.cloud.aiplatform import HyperparameterTuningJob, gapic, hyperparameter_tuning + from airflow.utils.context import Context + class CreateHyperparameterTuningJobOperator(GoogleCloudBaseOperator): """ diff --git a/providers/google/src/airflow/providers/google/cloud/operators/vertex_ai/model_service.py b/providers/google/src/airflow/providers/google/cloud/operators/vertex_ai/model_service.py index 4453ea284d0cf..79980685c0b14 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/vertex_ai/model_service.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/vertex_ai/model_service.py @@ -22,6 +22,10 @@ from collections.abc import Sequence from typing import TYPE_CHECKING +from google.api_core.exceptions import NotFound +from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault +from google.cloud.aiplatform_v1.types import Model, model_service + from airflow.providers.google.cloud.hooks.vertex_ai.model_service import ModelServiceHook from airflow.providers.google.cloud.links.vertex_ai import ( VertexAIModelExportLink, @@ -29,14 +33,12 @@ VertexAIModelListLink, ) from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator -from google.api_core.exceptions import NotFound -from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault -from google.cloud.aiplatform_v1.types import Model, model_service if TYPE_CHECKING: - from airflow.utils.context import Context from google.api_core.retry import Retry + from airflow.utils.context import Context + class DeleteModelOperator(GoogleCloudBaseOperator): """ diff --git a/providers/google/src/airflow/providers/google/cloud/operators/vertex_ai/pipeline_job.py b/providers/google/src/airflow/providers/google/cloud/operators/vertex_ai/pipeline_job.py index a68219e912abb..883dafe6a74eb 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/vertex_ai/pipeline_job.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/vertex_ai/pipeline_job.py @@ -23,6 +23,10 @@ from functools import cached_property from typing import TYPE_CHECKING, Any +from google.api_core.exceptions import NotFound +from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault +from google.cloud.aiplatform_v1 import types + from airflow.configuration import conf from airflow.exceptions import AirflowException from airflow.providers.google.cloud.hooks.vertex_ai.pipeline_job import PipelineJobHook @@ -32,16 +36,14 @@ ) from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator from airflow.providers.google.cloud.triggers.vertex_ai import RunPipelineJobTrigger -from google.api_core.exceptions import NotFound -from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault -from google.cloud.aiplatform_v1 import types if TYPE_CHECKING: - from airflow.utils.context import Context from google.api_core.retry import Retry from google.cloud.aiplatform import PipelineJob from google.cloud.aiplatform.metadata import experiment_resources + from airflow.utils.context import Context + class RunPipelineJobOperator(GoogleCloudBaseOperator): """ diff --git a/providers/google/src/airflow/providers/google/cloud/operators/video_intelligence.py b/providers/google/src/airflow/providers/google/cloud/operators/video_intelligence.py index de63fb492549e..0d4da307f63b8 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/video_intelligence.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/video_intelligence.py @@ -22,16 +22,18 @@ from collections.abc import Sequence from typing import TYPE_CHECKING -from airflow.providers.google.cloud.hooks.video_intelligence import CloudVideoIntelligenceHook -from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault from google.cloud.videointelligence_v1 import Feature, VideoContext from google.protobuf.json_format import MessageToDict +from airflow.providers.google.cloud.hooks.video_intelligence import CloudVideoIntelligenceHook +from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator + if TYPE_CHECKING: - from airflow.utils.context import Context from google.api_core.retry import Retry + from airflow.utils.context import Context + class CloudVideoIntelligenceDetectVideoLabelsOperator(GoogleCloudBaseOperator): """ diff --git a/providers/google/src/airflow/providers/google/cloud/operators/vision.py b/providers/google/src/airflow/providers/google/cloud/operators/vision.py index 3821b07d7f0d7..3f1c0bd74ce82 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/vision.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/vision.py @@ -23,9 +23,6 @@ from copy import deepcopy from typing import TYPE_CHECKING, Any -from airflow.providers.google.cloud.hooks.vision import CloudVisionHook -from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator -from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID from google.api_core.exceptions import AlreadyExists from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault from google.cloud.vision_v1 import ( @@ -36,11 +33,16 @@ ReferenceImage, ) +from airflow.providers.google.cloud.hooks.vision import CloudVisionHook +from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator +from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID + if TYPE_CHECKING: - from airflow.utils.context import Context from google.api_core.retry import Retry from google.protobuf.field_mask_pb2 import FieldMask + from airflow.utils.context import Context + MetaData = Sequence[tuple[str, str]] diff --git a/providers/google/src/airflow/providers/google/cloud/operators/workflows.py b/providers/google/src/airflow/providers/google/cloud/operators/workflows.py index 13ea8ba92dade..70b8eaef3950c 100644 --- a/providers/google/src/airflow/providers/google/cloud/operators/workflows.py +++ b/providers/google/src/airflow/providers/google/cloud/operators/workflows.py @@ -23,6 +23,11 @@ from collections.abc import Sequence from typing import TYPE_CHECKING +from google.api_core.exceptions import AlreadyExists +from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault +from google.cloud.workflows.executions_v1beta import Execution +from google.cloud.workflows_v1beta import Workflow + from airflow.providers.google.cloud.hooks.workflows import WorkflowsHook from airflow.providers.google.cloud.links.workflows import ( WorkflowsExecutionLink, @@ -31,16 +36,13 @@ ) from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID -from google.api_core.exceptions import AlreadyExists -from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault -from google.cloud.workflows.executions_v1beta import Execution -from google.cloud.workflows_v1beta import Workflow if TYPE_CHECKING: - from airflow.utils.context import Context from google.api_core.retry import Retry from google.protobuf.field_mask_pb2 import FieldMask + from airflow.utils.context import Context + from airflow.utils.hashlib_wrapper import md5 diff --git a/providers/google/src/airflow/providers/google/cloud/secrets/secret_manager.py b/providers/google/src/airflow/providers/google/cloud/secrets/secret_manager.py index c27ba8d57bf18..6f84b9a6f97fb 100644 --- a/providers/google/src/airflow/providers/google/cloud/secrets/secret_manager.py +++ b/providers/google/src/airflow/providers/google/cloud/secrets/secret_manager.py @@ -21,6 +21,8 @@ import logging from collections.abc import Sequence +from google.auth.exceptions import DefaultCredentialsError + 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 ( @@ -30,7 +32,6 @@ 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 -from google.auth.exceptions import DefaultCredentialsError log = logging.getLogger(__name__) diff --git a/providers/google/src/airflow/providers/google/cloud/sensors/bigquery_dts.py b/providers/google/src/airflow/providers/google/cloud/sensors/bigquery_dts.py index 27fedfa9b046e..96215849cdd1e 100644 --- a/providers/google/src/airflow/providers/google/cloud/sensors/bigquery_dts.py +++ b/providers/google/src/airflow/providers/google/cloud/sensors/bigquery_dts.py @@ -22,17 +22,19 @@ from collections.abc import Sequence from typing import TYPE_CHECKING +from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault +from google.cloud.bigquery_datatransfer_v1 import TransferState + from airflow.exceptions import AirflowException from airflow.providers.google.cloud.hooks.bigquery_dts import BiqQueryDataTransferServiceHook from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID from airflow.sensors.base import BaseSensorOperator -from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault -from google.cloud.bigquery_datatransfer_v1 import TransferState if TYPE_CHECKING: - from airflow.utils.context import Context from google.api_core.retry import Retry + from airflow.utils.context import Context + class BigQueryDataTransferServiceTransferRunSensor(BaseSensorOperator): """ diff --git a/providers/google/src/airflow/providers/google/cloud/sensors/bigtable.py b/providers/google/src/airflow/providers/google/cloud/sensors/bigtable.py index abe900f08b146..f67ebfc59a68c 100644 --- a/providers/google/src/airflow/providers/google/cloud/sensors/bigtable.py +++ b/providers/google/src/airflow/providers/google/cloud/sensors/bigtable.py @@ -23,13 +23,14 @@ from typing import TYPE_CHECKING import google.api_core.exceptions +from google.cloud.bigtable import enums +from google.cloud.bigtable.table import ClusterState + from airflow.providers.google.cloud.hooks.bigtable import BigtableHook from airflow.providers.google.cloud.links.bigtable import BigtableTablesLink from airflow.providers.google.cloud.operators.bigtable import BigtableValidationMixin from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID from airflow.sensors.base import BaseSensorOperator -from google.cloud.bigtable import enums -from google.cloud.bigtable.table import ClusterState if TYPE_CHECKING: from airflow.utils.context import Context diff --git a/providers/google/src/airflow/providers/google/cloud/sensors/cloud_composer.py b/providers/google/src/airflow/providers/google/cloud/sensors/cloud_composer.py index 2a195d015e79b..c59a97fe6fb1f 100644 --- a/providers/google/src/airflow/providers/google/cloud/sensors/cloud_composer.py +++ b/providers/google/src/airflow/providers/google/cloud/sensors/cloud_composer.py @@ -26,6 +26,7 @@ from typing import TYPE_CHECKING from dateutil import parser +from google.cloud.orchestration.airflow.service_v1.types import Environment, ExecuteAirflowCommandResponse from airflow.configuration import conf from airflow.exceptions import AirflowException @@ -34,7 +35,6 @@ from airflow.providers.google.common.consts import GOOGLE_DEFAULT_DEFERRABLE_METHOD_NAME from airflow.sensors.base import BaseSensorOperator from airflow.utils.state import TaskInstanceState -from google.cloud.orchestration.airflow.service_v1.types import Environment, ExecuteAirflowCommandResponse if TYPE_CHECKING: from airflow.utils.context import Context diff --git a/providers/google/src/airflow/providers/google/cloud/sensors/dataplex.py b/providers/google/src/airflow/providers/google/cloud/sensors/dataplex.py index a37b1684c0afb..b1ff0fdccf83b 100644 --- a/providers/google/src/airflow/providers/google/cloud/sensors/dataplex.py +++ b/providers/google/src/airflow/providers/google/cloud/sensors/dataplex.py @@ -23,9 +23,14 @@ from typing import TYPE_CHECKING if TYPE_CHECKING: - from airflow.utils.context import Context from google.api_core.retry import Retry + from airflow.utils.context import Context + +from google.api_core.exceptions import GoogleAPICallError +from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault +from google.cloud.dataplex_v1.types import DataScanJob + from airflow.exceptions import AirflowException from airflow.providers.google.cloud.hooks.dataplex import ( AirflowDataQualityScanException, @@ -33,9 +38,6 @@ DataplexHook, ) from airflow.sensors.base import BaseSensorOperator -from google.api_core.exceptions import GoogleAPICallError -from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault -from google.cloud.dataplex_v1.types import DataScanJob class TaskState: diff --git a/providers/google/src/airflow/providers/google/cloud/sensors/dataproc.py b/providers/google/src/airflow/providers/google/cloud/sensors/dataproc.py index 834f488554165..b8385c5b58b51 100644 --- a/providers/google/src/airflow/providers/google/cloud/sensors/dataproc.py +++ b/providers/google/src/airflow/providers/google/cloud/sensors/dataproc.py @@ -23,12 +23,13 @@ from collections.abc import Sequence from typing import TYPE_CHECKING +from google.api_core.exceptions import ServerError +from google.cloud.dataproc_v1.types import Batch, JobStatus + from airflow.exceptions import AirflowException from airflow.providers.google.cloud.hooks.dataproc import DataprocHook from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID from airflow.sensors.base import BaseSensorOperator -from google.api_core.exceptions import ServerError -from google.cloud.dataproc_v1.types import Batch, JobStatus if TYPE_CHECKING: from airflow.utils.context import Context diff --git a/providers/google/src/airflow/providers/google/cloud/sensors/dataproc_metastore.py b/providers/google/src/airflow/providers/google/cloud/sensors/dataproc_metastore.py index 731d29ee8f9d9..a3c712e783688 100644 --- a/providers/google/src/airflow/providers/google/cloud/sensors/dataproc_metastore.py +++ b/providers/google/src/airflow/providers/google/cloud/sensors/dataproc_metastore.py @@ -26,9 +26,10 @@ from airflow.sensors.base import BaseSensorOperator if TYPE_CHECKING: - from airflow.utils.context import Context from google.api_core.operation import Operation + from airflow.utils.context import Context + class MetastoreHivePartitionSensor(BaseSensorOperator): """ diff --git a/providers/google/src/airflow/providers/google/cloud/sensors/gcs.py b/providers/google/src/airflow/providers/google/cloud/sensors/gcs.py index e75ff2f7fd94e..569efea172c6c 100644 --- a/providers/google/src/airflow/providers/google/cloud/sensors/gcs.py +++ b/providers/google/src/airflow/providers/google/cloud/sensors/gcs.py @@ -25,6 +25,8 @@ from datetime import datetime, timedelta from typing import TYPE_CHECKING, Any, Callable +from google.cloud.storage.retry import DEFAULT_RETRY + from airflow.configuration import conf from airflow.exceptions import AirflowException from airflow.providers.google.cloud.hooks.gcs import GCSHook @@ -35,12 +37,12 @@ GCSUploadSessionTrigger, ) from airflow.sensors.base import BaseSensorOperator, poke_mode_only -from google.cloud.storage.retry import DEFAULT_RETRY if TYPE_CHECKING: - from airflow.utils.context import Context from google.api_core.retry import Retry + from airflow.utils.context import Context + class GCSObjectExistenceSensor(BaseSensorOperator): """ diff --git a/providers/google/src/airflow/providers/google/cloud/sensors/pubsub.py b/providers/google/src/airflow/providers/google/cloud/sensors/pubsub.py index 16200de88c206..03cf00f56695b 100644 --- a/providers/google/src/airflow/providers/google/cloud/sensors/pubsub.py +++ b/providers/google/src/airflow/providers/google/cloud/sensors/pubsub.py @@ -23,13 +23,14 @@ from datetime import timedelta from typing import TYPE_CHECKING, Any, Callable +from google.cloud import pubsub_v1 +from google.cloud.pubsub_v1.types import ReceivedMessage + from airflow.configuration import conf from airflow.exceptions import AirflowException from airflow.providers.google.cloud.hooks.pubsub import PubSubHook from airflow.providers.google.cloud.triggers.pubsub import PubsubPullTrigger from airflow.sensors.base import BaseSensorOperator -from google.cloud import pubsub_v1 -from google.cloud.pubsub_v1.types import ReceivedMessage if TYPE_CHECKING: from airflow.utils.context import Context diff --git a/providers/google/src/airflow/providers/google/cloud/sensors/workflows.py b/providers/google/src/airflow/providers/google/cloud/sensors/workflows.py index d62c1c73c7bb2..eeca15f0d76fb 100644 --- a/providers/google/src/airflow/providers/google/cloud/sensors/workflows.py +++ b/providers/google/src/airflow/providers/google/cloud/sensors/workflows.py @@ -19,17 +19,19 @@ from collections.abc import Sequence from typing import TYPE_CHECKING +from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault +from google.cloud.workflows.executions_v1beta import Execution + from airflow.exceptions import AirflowException from airflow.providers.google.cloud.hooks.workflows import WorkflowsHook from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID from airflow.sensors.base import BaseSensorOperator -from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault -from google.cloud.workflows.executions_v1beta import Execution if TYPE_CHECKING: - from airflow.utils.context import Context from google.api_core.retry import Retry + from airflow.utils.context import Context + class WorkflowExecutionSensor(BaseSensorOperator): """ diff --git a/providers/google/src/airflow/providers/google/cloud/transfers/bigquery_to_gcs.py b/providers/google/src/airflow/providers/google/cloud/transfers/bigquery_to_gcs.py index 950ed80461059..b005b79971f15 100644 --- a/providers/google/src/airflow/providers/google/cloud/transfers/bigquery_to_gcs.py +++ b/providers/google/src/airflow/providers/google/cloud/transfers/bigquery_to_gcs.py @@ -22,6 +22,9 @@ from collections.abc import Sequence from typing import TYPE_CHECKING, Any +from google.api_core.exceptions import Conflict +from google.cloud.bigquery import DEFAULT_RETRY, UnknownJob + from airflow.configuration import conf from airflow.exceptions import AirflowException from airflow.models import BaseOperator @@ -30,13 +33,12 @@ from airflow.providers.google.cloud.triggers.bigquery import BigQueryInsertJobTrigger from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID from airflow.utils.helpers import merge_dicts -from google.api_core.exceptions import Conflict -from google.cloud.bigquery import DEFAULT_RETRY, UnknownJob if TYPE_CHECKING: - from airflow.utils.context import Context from google.api_core.retry import Retry + from airflow.utils.context import Context + class BigQueryToGCSOperator(BaseOperator): """ diff --git a/providers/google/src/airflow/providers/google/cloud/transfers/gcs_to_bigquery.py b/providers/google/src/airflow/providers/google/cloud/transfers/gcs_to_bigquery.py index 83393ad28f59d..291608501b09d 100644 --- a/providers/google/src/airflow/providers/google/cloud/transfers/gcs_to_bigquery.py +++ b/providers/google/src/airflow/providers/google/cloud/transfers/gcs_to_bigquery.py @@ -23,15 +23,6 @@ from collections.abc import Sequence from typing import TYPE_CHECKING, Any -from airflow.configuration import conf -from airflow.exceptions import AirflowException -from airflow.models import BaseOperator -from airflow.providers.google.cloud.hooks.bigquery import BigQueryHook, BigQueryJob -from airflow.providers.google.cloud.hooks.gcs import GCSHook -from airflow.providers.google.cloud.links.bigquery import BigQueryTableLink -from airflow.providers.google.cloud.triggers.bigquery import BigQueryInsertJobTrigger -from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID -from airflow.utils.helpers import merge_dicts from google.api_core.exceptions import BadRequest, Conflict from google.cloud.bigquery import ( DEFAULT_RETRY, @@ -45,10 +36,21 @@ ) from google.cloud.bigquery.table import EncryptionConfiguration, Table, TableReference +from airflow.configuration import conf +from airflow.exceptions import AirflowException +from airflow.models import BaseOperator +from airflow.providers.google.cloud.hooks.bigquery import BigQueryHook, BigQueryJob +from airflow.providers.google.cloud.hooks.gcs import GCSHook +from airflow.providers.google.cloud.links.bigquery import BigQueryTableLink +from airflow.providers.google.cloud.triggers.bigquery import BigQueryInsertJobTrigger +from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID +from airflow.utils.helpers import merge_dicts + if TYPE_CHECKING: - from airflow.utils.context import Context from google.api_core.retry import Retry + from airflow.utils.context import Context + ALLOWED_FORMATS = [ "CSV", "NEWLINE_DELIMITED_JSON", diff --git a/providers/google/src/airflow/providers/google/cloud/triggers/bigquery_dts.py b/providers/google/src/airflow/providers/google/cloud/triggers/bigquery_dts.py index 18aaa1f278b77..6c99c8ef7cf2c 100644 --- a/providers/google/src/airflow/providers/google/cloud/triggers/bigquery_dts.py +++ b/providers/google/src/airflow/providers/google/cloud/triggers/bigquery_dts.py @@ -21,9 +21,10 @@ from collections.abc import AsyncIterator, Sequence from typing import Any +from google.cloud.bigquery_datatransfer_v1 import TransferRun, TransferState + from airflow.providers.google.cloud.hooks.bigquery_dts import AsyncBiqQueryDataTransferServiceHook from airflow.triggers.base import BaseTrigger, TriggerEvent -from google.cloud.bigquery_datatransfer_v1 import TransferRun, TransferState class BigQueryDataTransferRunTrigger(BaseTrigger): diff --git a/providers/google/src/airflow/providers/google/cloud/triggers/cloud_batch.py b/providers/google/src/airflow/providers/google/cloud/triggers/cloud_batch.py index ff921686bfee5..6e994ac80d779 100644 --- a/providers/google/src/airflow/providers/google/cloud/triggers/cloud_batch.py +++ b/providers/google/src/airflow/providers/google/cloud/triggers/cloud_batch.py @@ -20,9 +20,10 @@ from collections.abc import AsyncIterator, Sequence from typing import Any +from google.cloud.batch_v1 import Job, JobStatus + from airflow.providers.google.cloud.hooks.cloud_batch import CloudBatchAsyncHook from airflow.triggers.base import BaseTrigger, TriggerEvent -from google.cloud.batch_v1 import Job, JobStatus DEFAULT_BATCH_LOCATION = "us-central1" diff --git a/providers/google/src/airflow/providers/google/cloud/triggers/cloud_build.py b/providers/google/src/airflow/providers/google/cloud/triggers/cloud_build.py index 4e95a687c202f..5834d41ff7bb2 100644 --- a/providers/google/src/airflow/providers/google/cloud/triggers/cloud_build.py +++ b/providers/google/src/airflow/providers/google/cloud/triggers/cloud_build.py @@ -20,9 +20,10 @@ from collections.abc import AsyncIterator, Sequence from typing import Any +from google.cloud.devtools.cloudbuild_v1.types import Build + from airflow.providers.google.cloud.hooks.cloud_build import CloudBuildAsyncHook from airflow.triggers.base import BaseTrigger, TriggerEvent -from google.cloud.devtools.cloudbuild_v1.types import Build class CloudBuildCreateBuildTrigger(BaseTrigger): diff --git a/providers/google/src/airflow/providers/google/cloud/triggers/cloud_composer.py b/providers/google/src/airflow/providers/google/cloud/triggers/cloud_composer.py index 616b3a07e2d5a..96748bc2a7c49 100644 --- a/providers/google/src/airflow/providers/google/cloud/triggers/cloud_composer.py +++ b/providers/google/src/airflow/providers/google/cloud/triggers/cloud_composer.py @@ -25,11 +25,11 @@ from typing import Any from dateutil import parser +from google.cloud.orchestration.airflow.service_v1.types import ExecuteAirflowCommandResponse from airflow.exceptions import AirflowException from airflow.providers.google.cloud.hooks.cloud_composer import CloudComposerAsyncHook from airflow.triggers.base import BaseTrigger, TriggerEvent -from google.cloud.orchestration.airflow.service_v1.types import ExecuteAirflowCommandResponse class CloudComposerExecutionTrigger(BaseTrigger): diff --git a/providers/google/src/airflow/providers/google/cloud/triggers/cloud_storage_transfer_service.py b/providers/google/src/airflow/providers/google/cloud/triggers/cloud_storage_transfer_service.py index 7eae4bb697760..9a28b960e6e83 100644 --- a/providers/google/src/airflow/providers/google/cloud/triggers/cloud_storage_transfer_service.py +++ b/providers/google/src/airflow/providers/google/cloud/triggers/cloud_storage_transfer_service.py @@ -21,6 +21,9 @@ from collections.abc import AsyncIterator, Iterable, Sequence from typing import Any +from google.api_core.exceptions import GoogleAPIError +from google.cloud.storage_transfer_v1.types import TransferOperation + from airflow.exceptions import AirflowException from airflow.providers.google.cloud.hooks.cloud_storage_transfer_service import ( CloudDataTransferServiceAsyncHook, @@ -28,8 +31,6 @@ ) from airflow.providers.google.common.hooks.base_google import PROVIDE_PROJECT_ID from airflow.triggers.base import BaseTrigger, TriggerEvent -from google.api_core.exceptions import GoogleAPIError -from google.cloud.storage_transfer_v1.types import TransferOperation class CloudStorageTransferServiceCreateJobsTrigger(BaseTrigger): diff --git a/providers/google/src/airflow/providers/google/cloud/triggers/dataflow.py b/providers/google/src/airflow/providers/google/cloud/triggers/dataflow.py index 2a8eb6818dadd..e368949ce3d51 100644 --- a/providers/google/src/airflow/providers/google/cloud/triggers/dataflow.py +++ b/providers/google/src/airflow/providers/google/cloud/triggers/dataflow.py @@ -22,8 +22,6 @@ from functools import cached_property from typing import TYPE_CHECKING, Any -from airflow.providers.google.cloud.hooks.dataflow import AsyncDataflowHook, DataflowJobStatus -from airflow.triggers.base import BaseTrigger, TriggerEvent from google.cloud.dataflow_v1beta3 import JobState from google.cloud.dataflow_v1beta3.types import ( AutoscalingEvent, @@ -34,6 +32,9 @@ MetricUpdate, ) +from airflow.providers.google.cloud.hooks.dataflow import AsyncDataflowHook, DataflowJobStatus +from airflow.triggers.base import BaseTrigger, TriggerEvent + if TYPE_CHECKING: from google.cloud.dataflow_v1beta3.services.messages_v1_beta3.pagers import ListJobMessagesAsyncPager diff --git a/providers/google/src/airflow/providers/google/cloud/triggers/dataplex.py b/providers/google/src/airflow/providers/google/cloud/triggers/dataplex.py index eecd6187299d2..f9b83e004b571 100644 --- a/providers/google/src/airflow/providers/google/cloud/triggers/dataplex.py +++ b/providers/google/src/airflow/providers/google/cloud/triggers/dataplex.py @@ -22,9 +22,10 @@ import asyncio from collections.abc import AsyncIterator, Sequence +from google.cloud.dataplex_v1.types import DataScanJob + from airflow.providers.google.cloud.hooks.dataplex import DataplexAsyncHook from airflow.triggers.base import BaseTrigger, TriggerEvent -from google.cloud.dataplex_v1.types import DataScanJob class DataplexDataQualityJobTrigger(BaseTrigger): diff --git a/providers/google/src/airflow/providers/google/cloud/triggers/dataproc.py b/providers/google/src/airflow/providers/google/cloud/triggers/dataproc.py index 81f5ae86f7b2f..80afd381cc389 100644 --- a/providers/google/src/airflow/providers/google/cloud/triggers/dataproc.py +++ b/providers/google/src/airflow/providers/google/cloud/triggers/dataproc.py @@ -25,6 +25,9 @@ from collections.abc import AsyncIterator, Sequence from typing import TYPE_CHECKING, Any +from google.api_core.exceptions import NotFound +from google.cloud.dataproc_v1 import Batch, Cluster, ClusterStatus, JobStatus + from airflow.exceptions import AirflowException from airflow.models.taskinstance import TaskInstance from airflow.providers.google.cloud.hooks.dataproc import DataprocAsyncHook, DataprocHook @@ -33,8 +36,6 @@ from airflow.triggers.base import BaseTrigger, TriggerEvent from airflow.utils.session import provide_session from airflow.utils.state import TaskInstanceState -from google.api_core.exceptions import NotFound -from google.cloud.dataproc_v1 import Batch, Cluster, ClusterStatus, JobStatus if TYPE_CHECKING: from sqlalchemy.orm.session import Session diff --git a/providers/google/src/airflow/providers/google/cloud/triggers/kubernetes_engine.py b/providers/google/src/airflow/providers/google/cloud/triggers/kubernetes_engine.py index 171fb19ec7dc0..074ba568d944b 100644 --- a/providers/google/src/airflow/providers/google/cloud/triggers/kubernetes_engine.py +++ b/providers/google/src/airflow/providers/google/cloud/triggers/kubernetes_engine.py @@ -23,6 +23,7 @@ from functools import cached_property from typing import TYPE_CHECKING, Any +from google.cloud.container_v1.types import Operation from packaging.version import parse as parse_version from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning @@ -36,7 +37,6 @@ ) from airflow.providers_manager import ProvidersManager from airflow.triggers.base import BaseTrigger, TriggerEvent -from google.cloud.container_v1.types import Operation if TYPE_CHECKING: from datetime import datetime diff --git a/providers/google/src/airflow/providers/google/cloud/triggers/pubsub.py b/providers/google/src/airflow/providers/google/cloud/triggers/pubsub.py index d4bed2a18ed79..8b60801fd2d7c 100644 --- a/providers/google/src/airflow/providers/google/cloud/triggers/pubsub.py +++ b/providers/google/src/airflow/providers/google/cloud/triggers/pubsub.py @@ -23,9 +23,10 @@ from functools import cached_property from typing import Any +from google.cloud.pubsub_v1.types import ReceivedMessage + from airflow.providers.google.cloud.hooks.pubsub import PubSubAsyncHook from airflow.triggers.base import BaseTrigger, TriggerEvent -from google.cloud.pubsub_v1.types import ReceivedMessage class PubsubPullTrigger(BaseTrigger): diff --git a/providers/google/src/airflow/providers/google/cloud/triggers/vertex_ai.py b/providers/google/src/airflow/providers/google/cloud/triggers/vertex_ai.py index 29225bda44ec6..98c53411308ec 100644 --- a/providers/google/src/airflow/providers/google/cloud/triggers/vertex_ai.py +++ b/providers/google/src/airflow/providers/google/cloud/triggers/vertex_ai.py @@ -20,6 +20,14 @@ from functools import cached_property from typing import TYPE_CHECKING, Any +from google.cloud.aiplatform_v1 import ( + BatchPredictionJob, + HyperparameterTuningJob, + JobState, + PipelineState, + types, +) + from airflow.exceptions import AirflowException from airflow.providers.google.cloud.hooks.vertex_ai.batch_prediction_job import BatchPredictionJobAsyncHook from airflow.providers.google.cloud.hooks.vertex_ai.custom_job import CustomJobAsyncHook @@ -28,13 +36,6 @@ ) from airflow.providers.google.cloud.hooks.vertex_ai.pipeline_job import PipelineJobAsyncHook from airflow.triggers.base import BaseTrigger, TriggerEvent -from google.cloud.aiplatform_v1 import ( - BatchPredictionJob, - HyperparameterTuningJob, - JobState, - PipelineState, - types, -) if TYPE_CHECKING: from proto import Message diff --git a/providers/google/src/airflow/providers/google/cloud/utils/credentials_provider.py b/providers/google/src/airflow/providers/google/cloud/utils/credentials_provider.py index fab69d5c7c886..6be3fc3a0701c 100644 --- a/providers/google/src/airflow/providers/google/cloud/utils/credentials_provider.py +++ b/providers/google/src/airflow/providers/google/cloud/utils/credentials_provider.py @@ -29,6 +29,10 @@ import google.auth import google.oauth2.service_account +from google.auth import impersonated_credentials # type: ignore[attr-defined] +from google.auth.credentials import AnonymousCredentials, Credentials +from google.auth.environment_vars import CREDENTIALS, LEGACY_PROJECT, PROJECT + from airflow.exceptions import AirflowException from airflow.providers.google.cloud._internal_client.secret_manager_client import _SecretManagerClient from airflow.providers.google.cloud.utils.external_token_supplier import ( @@ -36,9 +40,6 @@ ) from airflow.utils.log.logging_mixin import LoggingMixin from airflow.utils.process_utils import patch_environ -from google.auth import impersonated_credentials # type: ignore[attr-defined] -from google.auth.credentials import AnonymousCredentials, Credentials -from google.auth.environment_vars import CREDENTIALS, LEGACY_PROJECT, PROJECT log = logging.getLogger(__name__) diff --git a/providers/google/src/airflow/providers/google/cloud/utils/external_token_supplier.py b/providers/google/src/airflow/providers/google/cloud/utils/external_token_supplier.py index 7666fcbe44603..6cd0caf1c12d7 100644 --- a/providers/google/src/airflow/providers/google/cloud/utils/external_token_supplier.py +++ b/providers/google/src/airflow/providers/google/cloud/utils/external_token_supplier.py @@ -22,7 +22,6 @@ from typing import TYPE_CHECKING, Any import requests - from google.auth.exceptions import RefreshError from google.auth.identity_pool import SubjectTokenSupplier diff --git a/providers/google/src/airflow/providers/google/common/auth_backend/google_openid.py b/providers/google/src/airflow/providers/google/common/auth_backend/google_openid.py index 7a47a89a94cd7..4a3f45fa4a3a8 100644 --- a/providers/google/src/airflow/providers/google/common/auth_backend/google_openid.py +++ b/providers/google/src/airflow/providers/google/common/auth_backend/google_openid.py @@ -23,17 +23,17 @@ from functools import wraps from typing import Callable, TypeVar, cast -from flask import Response, current_app, request as flask_request # type: ignore - import google import google.auth.transport.requests import google.oauth2.id_token -from airflow.configuration import conf -from airflow.providers.google.common.utils.id_token_credentials import get_default_id_token_credentials +from flask import Response, current_app, request as flask_request # type: ignore from google.auth import exceptions from google.auth.transport.requests import AuthorizedSession from google.oauth2 import service_account +from airflow.configuration import conf +from airflow.providers.google.common.utils.id_token_credentials import get_default_id_token_credentials + log = logging.getLogger(__name__) _GOOGLE_ISSUERS = ("accounts.google.com", "https://accounts.google.com") diff --git a/providers/google/src/airflow/providers/google/common/consts.py b/providers/google/src/airflow/providers/google/common/consts.py index 65408282ab52f..f8d7209901d73 100644 --- a/providers/google/src/airflow/providers/google/common/consts.py +++ b/providers/google/src/airflow/providers/google/common/consts.py @@ -16,9 +16,10 @@ # under the License. from __future__ import annotations -from airflow import version from google.api_core.gapic_v1.client_info import ClientInfo +from airflow import version + GOOGLE_DEFAULT_DEFERRABLE_METHOD_NAME = "execute_complete" CLIENT_INFO = ClientInfo(client_library_version="airflow_v" + version.version) diff --git a/providers/google/src/airflow/providers/google/common/hooks/base_google.py b/providers/google/src/airflow/providers/google/common/hooks/base_google.py index 756af502d4d9e..35bccf808cc72 100644 --- a/providers/google/src/airflow/providers/google/common/hooks/base_google.py +++ b/providers/google/src/airflow/providers/google/common/hooks/base_google.py @@ -31,18 +31,23 @@ from subprocess import check_output from typing import TYPE_CHECKING, Any, Callable, TypeVar, cast +import google.auth +import google.oauth2.service_account import google_auth_httplib2 import requests import tenacity from asgiref.sync import sync_to_async from gcloud.aio.auth.token import Token, TokenResponse +from google.api_core.exceptions import Forbidden, ResourceExhausted, TooManyRequests +from google.auth import _cloud_sdk, compute_engine # type: ignore[attr-defined] +from google.auth.environment_vars import CLOUD_SDK_CONFIG_DIR, CREDENTIALS +from google.auth.exceptions import RefreshError +from google.auth.transport import _http_client from googleapiclient import discovery from googleapiclient.errors import HttpError from googleapiclient.http import MediaIoBaseDownload, build_http, set_user_agent from requests import Session -import google.auth -import google.oauth2.service_account from airflow import version from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning from airflow.hooks.base import BaseHook @@ -54,15 +59,9 @@ from airflow.providers.google.common.consts import CLIENT_INFO from airflow.providers.google.common.deprecated import deprecated from airflow.utils.process_utils import patch_environ -from google.api_core.exceptions import Forbidden, ResourceExhausted, TooManyRequests -from google.auth import _cloud_sdk, compute_engine # type: ignore[attr-defined] -from google.auth.environment_vars import CLOUD_SDK_CONFIG_DIR, CREDENTIALS -from google.auth.exceptions import RefreshError -from google.auth.transport import _http_client if TYPE_CHECKING: from aiohttp import ClientSession - from google.api_core.gapic_v1.client_info import ClientInfo from google.auth.credentials import Credentials diff --git a/providers/google/src/airflow/providers/google/marketing_platform/hooks/analytics_admin.py b/providers/google/src/airflow/providers/google/marketing_platform/hooks/analytics_admin.py index ed65e97ca6f4a..492a91af2880f 100644 --- a/providers/google/src/airflow/providers/google/marketing_platform/hooks/analytics_admin.py +++ b/providers/google/src/airflow/providers/google/marketing_platform/hooks/analytics_admin.py @@ -30,8 +30,6 @@ from collections.abc import Sequence from typing import TYPE_CHECKING -from airflow.providers.google.common.consts import CLIENT_INFO -from airflow.providers.google.common.hooks.base_google import GoogleBaseHook from google.analytics.admin_v1beta import ( AnalyticsAdminServiceClient, DataStream, @@ -39,6 +37,9 @@ ) from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault +from airflow.providers.google.common.consts import CLIENT_INFO +from airflow.providers.google.common.hooks.base_google import GoogleBaseHook + if TYPE_CHECKING: from google.analytics.admin_v1beta.services.analytics_admin_service.pagers import ( ListAccountsPager, diff --git a/providers/google/src/airflow/providers/google/marketing_platform/hooks/search_ads.py b/providers/google/src/airflow/providers/google/marketing_platform/hooks/search_ads.py index 3d9387ea3a1e7..1d8be32e7cbab 100644 --- a/providers/google/src/airflow/providers/google/marketing_platform/hooks/search_ads.py +++ b/providers/google/src/airflow/providers/google/marketing_platform/hooks/search_ads.py @@ -23,11 +23,11 @@ from functools import cached_property from typing import TYPE_CHECKING, Any +from google.oauth2.credentials import Credentials from googleapiclient.discovery import build from airflow.exceptions import AirflowException from airflow.providers.google.common.hooks.base_google import GoogleBaseHook -from google.oauth2.credentials import Credentials if TYPE_CHECKING: from googleapiclient.discovery import Resource diff --git a/providers/google/src/airflow/providers/google/marketing_platform/operators/analytics_admin.py b/providers/google/src/airflow/providers/google/marketing_platform/operators/analytics_admin.py index 23fcc76fe3747..8f662f9c92d50 100644 --- a/providers/google/src/airflow/providers/google/marketing_platform/operators/analytics_admin.py +++ b/providers/google/src/airflow/providers/google/marketing_platform/operators/analytics_admin.py @@ -22,10 +22,6 @@ from collections.abc import Sequence from typing import TYPE_CHECKING, Any -from airflow.exceptions import AirflowNotFoundException -from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator -from airflow.providers.google.marketing_platform.hooks.analytics_admin import GoogleAnalyticsAdminHook -from airflow.providers.google.marketing_platform.links.analytics_admin import GoogleAnalyticsPropertyLink from google.analytics.admin_v1beta import ( Account, DataStream, @@ -34,11 +30,17 @@ ) from google.api_core.gapic_v1.method import DEFAULT, _MethodDefault +from airflow.exceptions import AirflowNotFoundException +from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator +from airflow.providers.google.marketing_platform.hooks.analytics_admin import GoogleAnalyticsAdminHook +from airflow.providers.google.marketing_platform.links.analytics_admin import GoogleAnalyticsPropertyLink + if TYPE_CHECKING: - from airflow.utils.context import Context from google.api_core.retry import Retry from google.protobuf.message import Message + from airflow.utils.context import Context + class GoogleAnalyticsAdminListAccountsOperator(GoogleCloudBaseOperator): """ diff --git a/providers/grpc/src/airflow/__init__.py b/providers/grpc/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/grpc/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/grpc/src/airflow/providers/__init__.py b/providers/grpc/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/grpc/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/grpc/src/airflow/providers/grpc/hooks/grpc.py b/providers/grpc/src/airflow/providers/grpc/hooks/grpc.py index 0da0857bbaa4c..2027f40b5834f 100644 --- a/providers/grpc/src/airflow/providers/grpc/hooks/grpc.py +++ b/providers/grpc/src/airflow/providers/grpc/hooks/grpc.py @@ -21,6 +21,7 @@ from collections.abc import Generator from typing import Any, Callable +import grpc from google import auth as google_auth from google.auth import jwt as google_auth_jwt # type: ignore[attr-defined] from google.auth.transport import ( @@ -28,7 +29,6 @@ requests as google_auth_transport_requests, ) -import grpc from airflow.exceptions import AirflowConfigException from airflow.hooks.base import BaseHook diff --git a/providers/hashicorp/src/airflow/__init__.py b/providers/hashicorp/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/hashicorp/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/hashicorp/src/airflow/providers/__init__.py b/providers/hashicorp/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/hashicorp/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/http/src/airflow/__init__.py b/providers/http/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/http/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/http/src/airflow/providers/__init__.py b/providers/http/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/http/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/imap/src/airflow/__init__.py b/providers/imap/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/imap/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/imap/src/airflow/providers/__init__.py b/providers/imap/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/imap/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/influxdb/src/airflow/__init__.py b/providers/influxdb/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/influxdb/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/influxdb/src/airflow/providers/__init__.py b/providers/influxdb/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/influxdb/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/jdbc/src/airflow/__init__.py b/providers/jdbc/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/jdbc/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/jdbc/src/airflow/providers/__init__.py b/providers/jdbc/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/jdbc/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/jenkins/src/airflow/__init__.py b/providers/jenkins/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/jenkins/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/jenkins/src/airflow/providers/__init__.py b/providers/jenkins/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/jenkins/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/jenkins/src/airflow/providers/jenkins/hooks/jenkins.py b/providers/jenkins/src/airflow/providers/jenkins/hooks/jenkins.py index a03eea2e04e87..e303b6d0ee072 100644 --- a/providers/jenkins/src/airflow/providers/jenkins/hooks/jenkins.py +++ b/providers/jenkins/src/airflow/providers/jenkins/hooks/jenkins.py @@ -20,6 +20,7 @@ from typing import Any import jenkins + from airflow.hooks.base import BaseHook diff --git a/providers/jenkins/src/airflow/providers/jenkins/operators/jenkins_job_trigger.py b/providers/jenkins/src/airflow/providers/jenkins/operators/jenkins_job_trigger.py index 43c979ba7ff88..218a3a7842c8b 100644 --- a/providers/jenkins/src/airflow/providers/jenkins/operators/jenkins_job_trigger.py +++ b/providers/jenkins/src/airflow/providers/jenkins/operators/jenkins_job_trigger.py @@ -26,13 +26,13 @@ from typing import Any, Union from urllib.error import HTTPError, URLError +import jenkins +from jenkins import Jenkins, JenkinsException from requests import Request -import jenkins from airflow.exceptions import AirflowException from airflow.models import BaseOperator from airflow.providers.jenkins.hooks.jenkins import JenkinsHook -from jenkins import Jenkins, JenkinsException JenkinsRequest = Mapping[str, Any] ParamType = Union[str, dict, list, None] diff --git a/providers/microsoft/azure/src/airflow/__init__.py b/providers/microsoft/azure/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/microsoft/azure/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/microsoft/azure/src/airflow/providers/__init__.py b/providers/microsoft/azure/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/microsoft/azure/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/microsoft/azure/src/airflow/providers/microsoft/__init__.py b/providers/microsoft/azure/src/airflow/providers/microsoft/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/microsoft/azure/src/airflow/providers/microsoft/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/fs/adls.py b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/fs/adls.py index 89bbd0281a963..84a242015f5b0 100644 --- a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/fs/adls.py +++ b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/fs/adls.py @@ -18,9 +18,10 @@ from typing import TYPE_CHECKING, Any +from azure.identity import ClientSecretCredential + from airflow.hooks.base import BaseHook from airflow.providers.microsoft.azure.utils import get_field, parse_blob_account_url -from azure.identity import ClientSecretCredential if TYPE_CHECKING: from fsspec import AbstractFileSystem diff --git a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/adx.py b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/adx.py index 7f7d38b2a80d7..e5ccc2342b626 100644 --- a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/adx.py +++ b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/adx.py @@ -30,14 +30,15 @@ from functools import cached_property from typing import TYPE_CHECKING, Any +from azure.kusto.data import ClientRequestProperties, KustoClient, KustoConnectionStringBuilder +from azure.kusto.data.exceptions import KustoServiceError + from airflow.exceptions import AirflowException from airflow.hooks.base import BaseHook from airflow.providers.microsoft.azure.utils import ( add_managed_identity_connection_widgets, get_sync_default_azure_credential, ) -from azure.kusto.data import ClientRequestProperties, KustoClient, KustoConnectionStringBuilder -from azure.kusto.data.exceptions import KustoServiceError if TYPE_CHECKING: from azure.kusto.data.response import KustoResponseDataSet diff --git a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/asb.py b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/asb.py index ac14622e5030a..d5cb666411e05 100644 --- a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/asb.py +++ b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/asb.py @@ -19,12 +19,6 @@ from typing import TYPE_CHECKING, Any, Callable from uuid import uuid4 -from airflow.hooks.base import BaseHook -from airflow.providers.microsoft.azure.utils import ( - add_managed_identity_connection_widgets, - get_field, - get_sync_default_azure_credential, -) from azure.core.exceptions import ResourceNotFoundError from azure.servicebus import ( ServiceBusClient, @@ -42,12 +36,20 @@ SubscriptionProperties, ) +from airflow.hooks.base import BaseHook +from airflow.providers.microsoft.azure.utils import ( + add_managed_identity_connection_widgets, + get_field, + get_sync_default_azure_credential, +) + if TYPE_CHECKING: import datetime - from airflow.utils.context import Context from azure.identity import DefaultAzureCredential + from airflow.utils.context import Context + MessageCallback = Callable[[ServiceBusMessage, Context], None] diff --git a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/base_azure.py b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/base_azure.py index ef247361c9969..c54ece1e041f6 100644 --- a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/base_azure.py +++ b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/base_azure.py @@ -18,14 +18,15 @@ from typing import Any +from azure.common.client_factory import get_client_from_auth_file, get_client_from_json_dict +from azure.common.credentials import ServicePrincipalCredentials + from airflow.exceptions import AirflowException from airflow.hooks.base import BaseHook from airflow.providers.microsoft.azure.utils import ( AzureIdentityCredentialAdapter, add_managed_identity_connection_widgets, ) -from azure.common.client_factory import get_client_from_auth_file, get_client_from_json_dict -from azure.common.credentials import ServicePrincipalCredentials class AzureBaseHook(BaseHook): diff --git a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/batch.py b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/batch.py index f2f062f460550..d4517fd170112 100644 --- a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/batch.py +++ b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/batch.py @@ -22,6 +22,8 @@ from functools import cached_property from typing import TYPE_CHECKING, Any +from azure.batch import BatchServiceClient, batch_auth, models as batch_models + from airflow.exceptions import AirflowException from airflow.hooks.base import BaseHook from airflow.providers.microsoft.azure.utils import ( @@ -30,7 +32,6 @@ get_field, ) from airflow.utils import timezone -from azure.batch import BatchServiceClient, batch_auth, models as batch_models if TYPE_CHECKING: from azure.batch.models import JobAddParameter, PoolAddParameter, TaskAddParameter diff --git a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/container_instance.py b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/container_instance.py index a4683cbf86238..4aaa668c5ae9f 100644 --- a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/container_instance.py +++ b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/container_instance.py @@ -20,13 +20,14 @@ from functools import cached_property from typing import TYPE_CHECKING, Any, cast -from airflow.exceptions import AirflowException -from airflow.providers.microsoft.azure.hooks.base_azure import AzureBaseHook -from airflow.providers.microsoft.azure.utils import get_sync_default_azure_credential from azure.common.client_factory import get_client_from_auth_file, get_client_from_json_dict from azure.identity import ClientSecretCredential, DefaultAzureCredential from azure.mgmt.containerinstance import ContainerInstanceManagementClient +from airflow.exceptions import AirflowException +from airflow.providers.microsoft.azure.hooks.base_azure import AzureBaseHook +from airflow.providers.microsoft.azure.utils import get_sync_default_azure_credential + if TYPE_CHECKING: from azure.mgmt.containerinstance.models import ( ContainerGroup, diff --git a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/container_registry.py b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/container_registry.py index d682f81852609..d860a5a567e95 100644 --- a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/container_registry.py +++ b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/container_registry.py @@ -22,14 +22,15 @@ from functools import cached_property from typing import Any +from azure.mgmt.containerinstance.models import ImageRegistryCredential +from azure.mgmt.containerregistry import ContainerRegistryManagementClient + from airflow.hooks.base import BaseHook from airflow.providers.microsoft.azure.utils import ( add_managed_identity_connection_widgets, get_field, get_sync_default_azure_credential, ) -from azure.mgmt.containerinstance.models import ImageRegistryCredential -from azure.mgmt.containerregistry import ContainerRegistryManagementClient class AzureContainerRegistryHook(BaseHook): diff --git a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/container_volume.py b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/container_volume.py index e394d2afab1c4..6e2700bf1e964 100644 --- a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/container_volume.py +++ b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/container_volume.py @@ -18,14 +18,15 @@ from typing import Any +from azure.mgmt.containerinstance.models import AzureFileVolume, Volume +from azure.mgmt.storage import StorageManagementClient + from airflow.hooks.base import BaseHook from airflow.providers.microsoft.azure.utils import ( add_managed_identity_connection_widgets, get_field, get_sync_default_azure_credential, ) -from azure.mgmt.containerinstance.models import AzureFileVolume, Volume -from azure.mgmt.storage import StorageManagementClient class AzureContainerVolumeHook(BaseHook): diff --git a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/cosmos.py b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/cosmos.py index 5cb79e5ee7274..da11ce10e0a16 100644 --- a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/cosmos.py +++ b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/cosmos.py @@ -30,6 +30,11 @@ from typing import TYPE_CHECKING, Any, Union from urllib.parse import urlparse +from azure.cosmos import PartitionKey +from azure.cosmos.cosmos_client import CosmosClient +from azure.cosmos.exceptions import CosmosHttpResponseError +from azure.mgmt.cosmosdb import CosmosDBManagementClient + from airflow.exceptions import AirflowBadRequest, AirflowException from airflow.hooks.base import BaseHook from airflow.providers.microsoft.azure.utils import ( @@ -37,10 +42,6 @@ get_field, get_sync_default_azure_credential, ) -from azure.cosmos import PartitionKey -from azure.cosmos.cosmos_client import CosmosClient -from azure.cosmos.exceptions import CosmosHttpResponseError -from azure.mgmt.cosmosdb import CosmosDBManagementClient if TYPE_CHECKING: PartitionKeyType = Union[str, list[str]] diff --git a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/data_factory.py b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/data_factory.py index a6815515ee309..2b12729f1a783 100644 --- a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/data_factory.py +++ b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/data_factory.py @@ -39,6 +39,13 @@ from typing import IO, TYPE_CHECKING, Any, Callable, TypeVar, Union, cast from asgiref.sync import sync_to_async +from azure.identity import ClientSecretCredential, DefaultAzureCredential +from azure.identity.aio import ( + ClientSecretCredential as AsyncClientSecretCredential, + DefaultAzureCredential as AsyncDefaultAzureCredential, +) +from azure.mgmt.datafactory import DataFactoryManagementClient +from azure.mgmt.datafactory.aio import DataFactoryManagementClient as AsyncDataFactoryManagementClient from airflow.exceptions import AirflowException from airflow.hooks.base import BaseHook @@ -47,13 +54,6 @@ get_async_default_azure_credential, get_sync_default_azure_credential, ) -from azure.identity import ClientSecretCredential, DefaultAzureCredential -from azure.identity.aio import ( - ClientSecretCredential as AsyncClientSecretCredential, - DefaultAzureCredential as AsyncDefaultAzureCredential, -) -from azure.mgmt.datafactory import DataFactoryManagementClient -from azure.mgmt.datafactory.aio import DataFactoryManagementClient as AsyncDataFactoryManagementClient if TYPE_CHECKING: from azure.core.polling import LROPoller diff --git a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/data_lake.py b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/data_lake.py index 0e0684f307a96..cc082677adf36 100644 --- a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/data_lake.py +++ b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/data_lake.py @@ -20,14 +20,6 @@ from functools import cached_property from typing import Any, Union -from airflow.exceptions import AirflowException -from airflow.hooks.base import BaseHook -from airflow.providers.microsoft.azure.utils import ( - AzureIdentityCredentialAdapter, - add_managed_identity_connection_widgets, - get_field, - get_sync_default_azure_credential, -) from azure.core.exceptions import ResourceExistsError, ResourceNotFoundError from azure.datalake.store import core, lib, multithread from azure.identity import ClientSecretCredential, DefaultAzureCredential @@ -40,6 +32,15 @@ FileSystemProperties, ) +from airflow.exceptions import AirflowException +from airflow.hooks.base import BaseHook +from airflow.providers.microsoft.azure.utils import ( + AzureIdentityCredentialAdapter, + add_managed_identity_connection_widgets, + get_field, + get_sync_default_azure_credential, +) + Credentials = Union[ClientSecretCredential, AzureIdentityCredentialAdapter, DefaultAzureCredential] diff --git a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/fileshare.py b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/fileshare.py index 78fba1f53a3e1..37c8647954d7d 100644 --- a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/fileshare.py +++ b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/fileshare.py @@ -19,12 +19,13 @@ from typing import IO, Any +from azure.storage.fileshare import FileProperties, ShareDirectoryClient, ShareFileClient, ShareServiceClient + from airflow.hooks.base import BaseHook from airflow.providers.microsoft.azure.utils import ( add_managed_identity_connection_widgets, get_sync_default_azure_credential, ) -from azure.storage.fileshare import FileProperties, ShareDirectoryClient, ShareFileClient, ShareServiceClient class AzureFileShareHook(BaseHook): diff --git a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/msgraph.py b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/msgraph.py index 01421b1b902fa..45a3c8f2c6f95 100644 --- a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/msgraph.py +++ b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/msgraph.py @@ -27,6 +27,7 @@ from urllib.parse import quote, urljoin, urlparse import httpx +from azure.identity import CertificateCredential, ClientSecretCredential from httpx import AsyncHTTPTransport, Timeout from kiota_abstractions.api_error import APIError from kiota_abstractions.method import Method @@ -50,9 +51,9 @@ AirflowNotFoundException, ) from airflow.hooks.base import BaseHook -from azure.identity import CertificateCredential, ClientSecretCredential if TYPE_CHECKING: + from azure.identity._internal.client_credential_base import ClientCredentialBase from kiota_abstractions.request_adapter import RequestAdapter from kiota_abstractions.request_information import QueryParams from kiota_abstractions.response_handler import NativeResponseType @@ -60,7 +61,6 @@ from kiota_http.httpx_request_adapter import ResponseType from airflow.models import Connection - from azure.identity._internal.client_credential_base import ClientCredentialBase class DefaultResponseHandler(ResponseHandler): diff --git a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/synapse.py b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/synapse.py index 81da92ad380d9..af5a77aebf07e 100644 --- a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/synapse.py +++ b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/synapse.py @@ -19,6 +19,11 @@ import time from typing import TYPE_CHECKING, Any, Union +from azure.core.exceptions import ServiceRequestError +from azure.identity import ClientSecretCredential, DefaultAzureCredential +from azure.synapse.artifacts import ArtifactsClient +from azure.synapse.spark import SparkClient + from airflow.exceptions import AirflowException, AirflowTaskTimeout from airflow.hooks.base import BaseHook from airflow.providers.microsoft.azure.utils import ( @@ -26,10 +31,6 @@ get_field, get_sync_default_azure_credential, ) -from azure.core.exceptions import ServiceRequestError -from azure.identity import ClientSecretCredential, DefaultAzureCredential -from azure.synapse.artifacts import ArtifactsClient -from azure.synapse.spark import SparkClient if TYPE_CHECKING: from azure.synapse.artifacts.models import CreateRunResponse, PipelineRun diff --git a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/wasb.py b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/wasb.py index a93f14beaf1fd..014be1903c146 100644 --- a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/wasb.py +++ b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/hooks/wasb.py @@ -32,15 +32,6 @@ from typing import TYPE_CHECKING, Any, Union from asgiref.sync import sync_to_async - -from airflow.exceptions import AirflowException -from airflow.hooks.base import BaseHook -from airflow.providers.microsoft.azure.utils import ( - add_managed_identity_connection_widgets, - get_async_default_azure_credential, - get_sync_default_azure_credential, - parse_blob_account_url, -) from azure.core.exceptions import HttpResponseError, ResourceExistsError, ResourceNotFoundError from azure.identity import ClientSecretCredential from azure.identity.aio import ( @@ -54,6 +45,15 @@ ContainerClient as AsyncContainerClient, ) +from airflow.exceptions import AirflowException +from airflow.hooks.base import BaseHook +from airflow.providers.microsoft.azure.utils import ( + add_managed_identity_connection_widgets, + get_async_default_azure_credential, + get_sync_default_azure_credential, + parse_blob_account_url, +) + if TYPE_CHECKING: from azure.storage.blob._models import BlobProperties diff --git a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/log/wasb_task_handler.py b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/log/wasb_task_handler.py index 010631f5a83cd..d32a3463d74cd 100644 --- a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/log/wasb_task_handler.py +++ b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/log/wasb_task_handler.py @@ -23,10 +23,11 @@ from pathlib import Path from typing import TYPE_CHECKING +from azure.core.exceptions import HttpResponseError + from airflow.configuration import conf from airflow.utils.log.file_task_handler import FileTaskHandler from airflow.utils.log.logging_mixin import LoggingMixin -from azure.core.exceptions import HttpResponseError if TYPE_CHECKING: import logging diff --git a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/operators/adx.py b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/operators/adx.py index dbecb522dcf52..05c2cd8d54eeb 100644 --- a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/operators/adx.py +++ b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/operators/adx.py @@ -28,9 +28,10 @@ from airflow.providers.microsoft.azure.hooks.adx import AzureDataExplorerHook if TYPE_CHECKING: - from airflow.utils.context import Context from azure.kusto.data._models import KustoResultTable + from airflow.utils.context import Context + class AzureDataExplorerQueryOperator(BaseOperator): """ diff --git a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/operators/asb.py b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/operators/asb.py index 056c6b1498717..aa8eecb3f2442 100644 --- a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/operators/asb.py +++ b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/operators/asb.py @@ -25,10 +25,11 @@ if TYPE_CHECKING: import datetime - from airflow.utils.context import Context from azure.servicebus import ServiceBusMessage from azure.servicebus.management import AuthorizationRule, CorrelationRuleFilter, SqlRuleFilter + from airflow.utils.context import Context + MessageCallback = Callable[[ServiceBusMessage, Context], None] diff --git a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/operators/batch.py b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/operators/batch.py index 10dbe8dad53af..5b366b5b6bdc0 100644 --- a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/operators/batch.py +++ b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/operators/batch.py @@ -21,10 +21,11 @@ from functools import cached_property from typing import TYPE_CHECKING, Any +from azure.batch import models as batch_models + from airflow.exceptions import AirflowException from airflow.models import BaseOperator from airflow.providers.microsoft.azure.hooks.batch import AzureBatchHook -from azure.batch import models as batch_models if TYPE_CHECKING: from airflow.utils.context import Context diff --git a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/operators/container_instances.py b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/operators/container_instances.py index 746a8d5566812..8b21f8226b3ac 100644 --- a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/operators/container_instances.py +++ b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/operators/container_instances.py @@ -23,13 +23,6 @@ from collections.abc import Sequence from typing import TYPE_CHECKING, Any -from msrestazure.azure_exceptions import CloudError - -from airflow.exceptions import AirflowException, AirflowTaskTimeout -from airflow.models import BaseOperator -from airflow.providers.microsoft.azure.hooks.container_instance import AzureContainerInstanceHook -from airflow.providers.microsoft.azure.hooks.container_registry import AzureContainerRegistryHook -from airflow.providers.microsoft.azure.hooks.container_volume import AzureContainerVolumeHook from azure.mgmt.containerinstance.models import ( Container, ContainerGroup, @@ -44,6 +37,13 @@ Volume as _AzureVolume, VolumeMount, ) +from msrestazure.azure_exceptions import CloudError + +from airflow.exceptions import AirflowException, AirflowTaskTimeout +from airflow.models import BaseOperator +from airflow.providers.microsoft.azure.hooks.container_instance import AzureContainerInstanceHook +from airflow.providers.microsoft.azure.hooks.container_registry import AzureContainerRegistryHook +from airflow.providers.microsoft.azure.hooks.container_volume import AzureContainerVolumeHook if TYPE_CHECKING: from airflow.utils.context import Context diff --git a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/operators/synapse.py b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/operators/synapse.py index 534d82446c221..275a45b30f455 100644 --- a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/operators/synapse.py +++ b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/operators/synapse.py @@ -33,9 +33,10 @@ ) if TYPE_CHECKING: + from azure.synapse.spark.models import SparkBatchJobOptions + from airflow.models.taskinstancekey import TaskInstanceKey from airflow.utils.context import Context - from azure.synapse.spark.models import SparkBatchJobOptions class AzureSynapseRunSparkBatchOperator(BaseOperator): diff --git a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/secrets/key_vault.py b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/secrets/key_vault.py index 841ab0b6f54ea..aa1724b8e1bdb 100644 --- a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/secrets/key_vault.py +++ b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/secrets/key_vault.py @@ -28,13 +28,14 @@ import os from functools import cached_property -from airflow.providers.microsoft.azure.utils import get_sync_default_azure_credential -from airflow.secrets import BaseSecretsBackend -from airflow.utils.log.logging_mixin import LoggingMixin from azure.core.exceptions import ResourceNotFoundError from azure.identity import ClientSecretCredential, DefaultAzureCredential from azure.keyvault.secrets import SecretClient +from airflow.providers.microsoft.azure.utils import get_sync_default_azure_credential +from airflow.secrets import BaseSecretsBackend +from airflow.utils.log.logging_mixin import LoggingMixin + class AzureKeyVaultBackend(BaseSecretsBackend, LoggingMixin): """ diff --git a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/triggers/data_factory.py b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/triggers/data_factory.py index 7d9f3208414d3..5e275a66762ef 100644 --- a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/triggers/data_factory.py +++ b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/triggers/data_factory.py @@ -21,12 +21,13 @@ from collections.abc import AsyncIterator from typing import Any +from azure.core.exceptions import ServiceRequestError + from airflow.providers.microsoft.azure.hooks.data_factory import ( AzureDataFactoryAsyncHook, AzureDataFactoryPipelineRunStatus, ) from airflow.triggers.base import BaseTrigger, TriggerEvent -from azure.core.exceptions import ServiceRequestError class ADFPipelineRunStatusSensorTrigger(BaseTrigger): diff --git a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/utils.py b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/utils.py index 7c88acaf72232..faea3b7ba4b52 100644 --- a/providers/microsoft/azure/src/airflow/providers/microsoft/azure/utils.py +++ b/providers/microsoft/azure/src/airflow/providers/microsoft/azure/utils.py @@ -21,13 +21,12 @@ from functools import partial, wraps from urllib.parse import urlparse, urlunparse -from msrest.authentication import BasicTokenAuthentication - from azure.core.pipeline import PipelineContext, PipelineRequest from azure.core.pipeline.policies import BearerTokenCredentialPolicy from azure.core.pipeline.transport import HttpRequest from azure.identity import DefaultAzureCredential from azure.identity.aio import DefaultAzureCredential as AsyncDefaultAzureCredential +from msrest.authentication import BasicTokenAuthentication def get_field(*, conn_id: str, conn_type: str, extras: dict, field_name: str): diff --git a/providers/microsoft/mssql/src/airflow/__init__.py b/providers/microsoft/mssql/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/microsoft/mssql/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/microsoft/mssql/src/airflow/providers/__init__.py b/providers/microsoft/mssql/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/microsoft/mssql/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/microsoft/mssql/src/airflow/providers/microsoft/__init__.py b/providers/microsoft/mssql/src/airflow/providers/microsoft/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/microsoft/mssql/src/airflow/providers/microsoft/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/microsoft/psrp/src/airflow/__init__.py b/providers/microsoft/psrp/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/microsoft/psrp/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/microsoft/psrp/src/airflow/providers/__init__.py b/providers/microsoft/psrp/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/microsoft/psrp/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/microsoft/psrp/src/airflow/providers/microsoft/__init__.py b/providers/microsoft/psrp/src/airflow/providers/microsoft/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/microsoft/psrp/src/airflow/providers/microsoft/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/microsoft/winrm/src/airflow/__init__.py b/providers/microsoft/winrm/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/microsoft/winrm/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/microsoft/winrm/src/airflow/providers/__init__.py b/providers/microsoft/winrm/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/microsoft/winrm/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/microsoft/winrm/src/airflow/providers/microsoft/__init__.py b/providers/microsoft/winrm/src/airflow/providers/microsoft/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/microsoft/winrm/src/airflow/providers/microsoft/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/microsoft/winrm/src/airflow/providers/microsoft/winrm/hooks/winrm.py b/providers/microsoft/winrm/src/airflow/providers/microsoft/winrm/hooks/winrm.py index 4ded4cdffacf2..961e37ba3fe40 100644 --- a/providers/microsoft/winrm/src/airflow/providers/microsoft/winrm/hooks/winrm.py +++ b/providers/microsoft/winrm/src/airflow/providers/microsoft/winrm/hooks/winrm.py @@ -22,11 +22,12 @@ from base64 import b64encode from contextlib import suppress +from winrm.exceptions import WinRMOperationTimeoutError +from winrm.protocol import Protocol + from airflow.exceptions import AirflowException from airflow.hooks.base import BaseHook from airflow.utils.platform import getuser -from winrm.exceptions import WinRMOperationTimeoutError -from winrm.protocol import Protocol # TODO: FIXME please - I have too complex implementation diff --git a/providers/mongo/src/airflow/__init__.py b/providers/mongo/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/mongo/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/mongo/src/airflow/providers/__init__.py b/providers/mongo/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/mongo/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/mysql/src/airflow/__init__.py b/providers/mysql/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/mysql/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/mysql/src/airflow/providers/__init__.py b/providers/mysql/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/mysql/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/neo4j/src/airflow/__init__.py b/providers/neo4j/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/neo4j/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/neo4j/src/airflow/providers/__init__.py b/providers/neo4j/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/neo4j/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/neo4j/src/airflow/providers/neo4j/hooks/neo4j.py b/providers/neo4j/src/airflow/providers/neo4j/hooks/neo4j.py index 5c8b13857a6b9..5f58f802086f9 100644 --- a/providers/neo4j/src/airflow/providers/neo4j/hooks/neo4j.py +++ b/providers/neo4j/src/airflow/providers/neo4j/hooks/neo4j.py @@ -22,9 +22,10 @@ from typing import TYPE_CHECKING, Any from urllib.parse import urlsplit -from airflow.hooks.base import BaseHook from neo4j import Driver, GraphDatabase +from airflow.hooks.base import BaseHook + if TYPE_CHECKING: from airflow.models import Connection diff --git a/providers/odbc/src/airflow/__init__.py b/providers/odbc/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/odbc/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/odbc/src/airflow/providers/__init__.py b/providers/odbc/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/odbc/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/openai/src/airflow/__init__.py b/providers/openai/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/openai/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/openai/src/airflow/providers/__init__.py b/providers/openai/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/openai/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/openfaas/src/airflow/__init__.py b/providers/openfaas/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/openfaas/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/openfaas/src/airflow/providers/__init__.py b/providers/openfaas/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/openfaas/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/openlineage/src/airflow/__init__.py b/providers/openlineage/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/openlineage/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/openlineage/src/airflow/providers/__init__.py b/providers/openlineage/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/openlineage/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/openlineage/src/airflow/providers/openlineage/extractors/base.py b/providers/openlineage/src/airflow/providers/openlineage/extractors/base.py index 1d2ec4449dd3c..2b85825d8c6ee 100644 --- a/providers/openlineage/src/airflow/providers/openlineage/extractors/base.py +++ b/providers/openlineage/src/airflow/providers/openlineage/extractors/base.py @@ -22,16 +22,16 @@ from typing import Generic, TypeVar, Union from attrs import Factory, define - from openlineage.client.event_v2 import Dataset as OLDataset with warnings.catch_warnings(): warnings.simplefilter("ignore", DeprecationWarning) from openlineage.client.facet import BaseFacet as BaseFacet_V1 +from openlineage.client.facet_v2 import JobFacet, RunFacet + from airflow.providers.openlineage.utils.utils import AIRFLOW_V_2_10_PLUS from airflow.utils.log.logging_mixin import LoggingMixin from airflow.utils.state import TaskInstanceState -from openlineage.client.facet_v2 import JobFacet, RunFacet # this is not to break static checks compatibility with v1 OpenLineage facet classes DatasetSubclass = TypeVar("DatasetSubclass", bound=OLDataset) diff --git a/providers/openlineage/src/airflow/providers/openlineage/extractors/bash.py b/providers/openlineage/src/airflow/providers/openlineage/extractors/bash.py index 42d3573b7b89d..6e1b3f28eefe0 100644 --- a/providers/openlineage/src/airflow/providers/openlineage/extractors/bash.py +++ b/providers/openlineage/src/airflow/providers/openlineage/extractors/bash.py @@ -17,10 +17,11 @@ from __future__ import annotations +from openlineage.client.facet_v2 import source_code_job + from airflow.providers.openlineage import conf from airflow.providers.openlineage.extractors.base import BaseExtractor, OperatorLineage from airflow.providers.openlineage.utils.utils import get_unknown_source_attribute_run_facet -from openlineage.client.facet_v2 import source_code_job """ :meta private: diff --git a/providers/openlineage/src/airflow/providers/openlineage/extractors/manager.py b/providers/openlineage/src/airflow/providers/openlineage/extractors/manager.py index 618a78bb979b3..f07014885eaff 100644 --- a/providers/openlineage/src/airflow/providers/openlineage/extractors/manager.py +++ b/providers/openlineage/src/airflow/providers/openlineage/extractors/manager.py @@ -34,9 +34,10 @@ from airflow.utils.log.logging_mixin import LoggingMixin if TYPE_CHECKING: + from openlineage.client.event_v2 import Dataset + from airflow.models import Operator from airflow.providers.common.compat.lineage.entities import Table - from openlineage.client.event_v2 import Dataset def _iter_extractor_types() -> Iterator[type[BaseExtractor]]: @@ -290,9 +291,10 @@ def convert_to_ol_dataset_from_table(table: Table) -> Dataset: @staticmethod def convert_to_ol_dataset(obj) -> Dataset | None: - from airflow.providers.common.compat.lineage.entities import File, Table from openlineage.client.event_v2 import Dataset + from airflow.providers.common.compat.lineage.entities import File, Table + if isinstance(obj, Dataset): return obj elif isinstance(obj, Table): diff --git a/providers/openlineage/src/airflow/providers/openlineage/extractors/python.py b/providers/openlineage/src/airflow/providers/openlineage/extractors/python.py index 8d50e74bb5684..c716e28b4d8f0 100644 --- a/providers/openlineage/src/airflow/providers/openlineage/extractors/python.py +++ b/providers/openlineage/src/airflow/providers/openlineage/extractors/python.py @@ -20,10 +20,11 @@ import inspect from typing import Callable +from openlineage.client.facet_v2 import source_code_job + from airflow.providers.openlineage import conf from airflow.providers.openlineage.extractors.base import BaseExtractor, OperatorLineage from airflow.providers.openlineage.utils.utils import get_unknown_source_attribute_run_facet -from openlineage.client.facet_v2 import source_code_job """ :meta private: diff --git a/providers/openlineage/src/airflow/providers/openlineage/plugins/adapter.py b/providers/openlineage/src/airflow/providers/openlineage/plugins/adapter.py index 0e6a519271f07..8350b2a0d517c 100644 --- a/providers/openlineage/src/airflow/providers/openlineage/plugins/adapter.py +++ b/providers/openlineage/src/airflow/providers/openlineage/plugins/adapter.py @@ -21,16 +21,6 @@ from typing import TYPE_CHECKING import yaml - -from airflow.providers.openlineage import __version__ as OPENLINEAGE_PROVIDER_VERSION, conf -from airflow.providers.openlineage.utils.utils import ( - OpenLineageRedactor, - get_airflow_debug_facet, - get_airflow_state_run_facet, - get_processing_engine_facet, -) -from airflow.stats import Stats -from airflow.utils.log.logging_mixin import LoggingMixin from openlineage.client import OpenLineageClient, set_producer from openlineage.client.event_v2 import Job, Run, RunEvent, RunState from openlineage.client.facet_v2 import ( @@ -46,6 +36,16 @@ ) from openlineage.client.uuid import generate_static_uuid +from airflow.providers.openlineage import __version__ as OPENLINEAGE_PROVIDER_VERSION, conf +from airflow.providers.openlineage.utils.utils import ( + OpenLineageRedactor, + get_airflow_debug_facet, + get_airflow_state_run_facet, + get_processing_engine_facet, +) +from airflow.stats import Stats +from airflow.utils.log.logging_mixin import LoggingMixin + if TYPE_CHECKING: from datetime import datetime diff --git a/providers/openlineage/src/airflow/providers/openlineage/plugins/facets.py b/providers/openlineage/src/airflow/providers/openlineage/plugins/facets.py index 3c3f1d9690e4f..e35d312b70667 100644 --- a/providers/openlineage/src/airflow/providers/openlineage/plugins/facets.py +++ b/providers/openlineage/src/airflow/providers/openlineage/plugins/facets.py @@ -17,7 +17,6 @@ from __future__ import annotations from attrs import define - from openlineage.client.facet_v2 import JobFacet, RunFacet from openlineage.client.utils import RedactMixin diff --git a/providers/openlineage/src/airflow/providers/openlineage/plugins/listener.py b/providers/openlineage/src/airflow/providers/openlineage/plugins/listener.py index 378f9f00bbfb5..7ddd0a7ec7e75 100644 --- a/providers/openlineage/src/airflow/providers/openlineage/plugins/listener.py +++ b/providers/openlineage/src/airflow/providers/openlineage/plugins/listener.py @@ -23,6 +23,7 @@ from typing import TYPE_CHECKING import psutil +from openlineage.client.serde import Serde from setproctitle import getproctitle, setproctitle from airflow import settings @@ -50,7 +51,6 @@ from airflow.utils import timezone from airflow.utils.state import TaskInstanceState from airflow.utils.timeout import timeout -from openlineage.client.serde import Serde if TYPE_CHECKING: from airflow.models import TaskInstance diff --git a/providers/openlineage/src/airflow/providers/openlineage/sqlparser.py b/providers/openlineage/src/airflow/providers/openlineage/sqlparser.py index 7edb01cabd22a..d2d37b317355f 100644 --- a/providers/openlineage/src/airflow/providers/openlineage/sqlparser.py +++ b/providers/openlineage/src/airflow/providers/openlineage/sqlparser.py @@ -21,6 +21,9 @@ import sqlparse from attrs import define +from openlineage.client.event_v2 import Dataset +from openlineage.client.facet_v2 import column_lineage_dataset, extraction_error_run, sql_job +from openlineage.common.sql import DbTableMeta, SqlMeta, parse from airflow.providers.openlineage.extractors.base import OperatorLineage from airflow.providers.openlineage.utils.sql import ( @@ -30,16 +33,13 @@ ) from airflow.providers.openlineage.utils.utils import should_use_external_connection from airflow.utils.log.logging_mixin import LoggingMixin -from openlineage.client.event_v2 import Dataset -from openlineage.client.facet_v2 import column_lineage_dataset, extraction_error_run, sql_job -from openlineage.common.sql import DbTableMeta, SqlMeta, parse if TYPE_CHECKING: + from openlineage.client.facet_v2 import JobFacet, RunFacet from sqlalchemy.engine import Engine from airflow.hooks.base import BaseHook from airflow.providers.common.sql.hooks.sql import DbApiHook - from openlineage.client.facet_v2 import JobFacet, RunFacet log = logging.getLogger(__name__) diff --git a/providers/openlineage/src/airflow/providers/openlineage/utils/sql.py b/providers/openlineage/src/airflow/providers/openlineage/utils/sql.py index acc29f3a7db01..1223a3618bd7b 100644 --- a/providers/openlineage/src/airflow/providers/openlineage/utils/sql.py +++ b/providers/openlineage/src/airflow/providers/openlineage/utils/sql.py @@ -23,10 +23,9 @@ from typing import TYPE_CHECKING, Optional from attrs import define -from sqlalchemy import Column, MetaData, Table, and_, or_, union_all - from openlineage.client.event_v2 import Dataset from openlineage.client.facet_v2 import schema_dataset +from sqlalchemy import Column, MetaData, Table, and_, or_, union_all if TYPE_CHECKING: from sqlalchemy.engine import Engine diff --git a/providers/openlineage/src/airflow/providers/openlineage/utils/utils.py b/providers/openlineage/src/airflow/providers/openlineage/utils/utils.py index 336992ff29bb7..852d9d731f8a3 100644 --- a/providers/openlineage/src/airflow/providers/openlineage/utils/utils.py +++ b/providers/openlineage/src/airflow/providers/openlineage/utils/utils.py @@ -26,6 +26,7 @@ from typing import TYPE_CHECKING, Any, Callable import attrs +from openlineage.client.utils import RedactMixin from airflow import __version__ as AIRFLOW_VERSION @@ -54,7 +55,6 @@ from airflow.serialization.serialized_objects import SerializedBaseOperator from airflow.utils.module_loading import import_string from airflow.utils.session import NEW_SESSION, provide_session -from openlineage.client.utils import RedactMixin try: from airflow.sdk import BaseOperator as SdkBaseOperator @@ -62,6 +62,9 @@ SdkBaseOperator = BaseOperator # type: ignore[misc] if TYPE_CHECKING: + from openlineage.client.event_v2 import Dataset as OpenLineageDataset + from openlineage.client.facet_v2 import RunFacet, processing_engine_run + from airflow.models import TaskInstance from airflow.providers.common.compat.assets import Asset from airflow.sdk import DAG, MappedOperator @@ -72,8 +75,6 @@ should_hide_value_for_key, ) from airflow.utils.state import DagRunState, TaskInstanceState - from openlineage.client.event_v2 import Dataset as OpenLineageDataset - from openlineage.client.facet_v2 import RunFacet, processing_engine_run else: try: from airflow.sdk import DAG, MappedOperator diff --git a/providers/opensearch/src/airflow/__init__.py b/providers/opensearch/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/opensearch/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/opensearch/src/airflow/providers/__init__.py b/providers/opensearch/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/opensearch/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/opsgenie/src/airflow/__init__.py b/providers/opsgenie/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/opsgenie/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/opsgenie/src/airflow/providers/__init__.py b/providers/opsgenie/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/opsgenie/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/oracle/src/airflow/__init__.py b/providers/oracle/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/oracle/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/oracle/src/airflow/providers/__init__.py b/providers/oracle/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/oracle/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/pagerduty/src/airflow/__init__.py b/providers/pagerduty/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/pagerduty/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/pagerduty/src/airflow/providers/__init__.py b/providers/pagerduty/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/pagerduty/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/papermill/src/airflow/__init__.py b/providers/papermill/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/papermill/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/papermill/src/airflow/providers/__init__.py b/providers/papermill/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/papermill/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/papermill/src/airflow/providers/papermill/hooks/kernel.py b/providers/papermill/src/airflow/providers/papermill/hooks/kernel.py index dcc5caedf0e71..60c4edc6fd7f6 100644 --- a/providers/papermill/src/airflow/providers/papermill/hooks/kernel.py +++ b/providers/papermill/src/airflow/providers/papermill/hooks/kernel.py @@ -19,12 +19,12 @@ import typing from jupyter_client import AsyncKernelManager -from traitlets import Unicode - -from airflow.hooks.base import BaseHook from papermill.clientwrap import PapermillNotebookClient from papermill.engines import NBClientEngine from papermill.utils import merge_kwargs, remove_args +from traitlets import Unicode + +from airflow.hooks.base import BaseHook JUPYTER_KERNEL_SHELL_PORT = 60316 JUPYTER_KERNEL_IOPUB_PORT = 60317 diff --git a/providers/papermill/src/airflow/providers/papermill/operators/papermill.py b/providers/papermill/src/airflow/providers/papermill/operators/papermill.py index fe95c4089309b..ddb7a92541231 100644 --- a/providers/papermill/src/airflow/providers/papermill/operators/papermill.py +++ b/providers/papermill/src/airflow/providers/papermill/operators/papermill.py @@ -22,8 +22,8 @@ from typing import TYPE_CHECKING, ClassVar import attr - import papermill as pm + from airflow.models import BaseOperator from airflow.providers.common.compat.lineage.entities import File from airflow.providers.papermill.hooks.kernel import REMOTE_KERNEL_ENGINE, KernelHook diff --git a/providers/pgvector/src/airflow/__init__.py b/providers/pgvector/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/pgvector/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/pgvector/src/airflow/providers/__init__.py b/providers/pgvector/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/pgvector/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/pgvector/src/airflow/providers/pgvector/operators/pgvector.py b/providers/pgvector/src/airflow/providers/pgvector/operators/pgvector.py index c9f1daed1fea7..ae48a377ea7a7 100644 --- a/providers/pgvector/src/airflow/providers/pgvector/operators/pgvector.py +++ b/providers/pgvector/src/airflow/providers/pgvector/operators/pgvector.py @@ -17,9 +17,10 @@ # under the License. from __future__ import annotations -from airflow.providers.common.sql.operators.sql import SQLExecuteQueryOperator from pgvector.psycopg2 import register_vector +from airflow.providers.common.sql.operators.sql import SQLExecuteQueryOperator + class PgVectorIngestOperator(SQLExecuteQueryOperator): """ diff --git a/providers/pinecone/src/airflow/__init__.py b/providers/pinecone/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/pinecone/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/pinecone/src/airflow/providers/__init__.py b/providers/pinecone/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/pinecone/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/pinecone/src/airflow/providers/pinecone/hooks/pinecone.py b/providers/pinecone/src/airflow/providers/pinecone/hooks/pinecone.py index 14bb31b6983ca..932fd04acd6ea 100644 --- a/providers/pinecone/src/airflow/providers/pinecone/hooks/pinecone.py +++ b/providers/pinecone/src/airflow/providers/pinecone/hooks/pinecone.py @@ -24,15 +24,17 @@ from functools import cached_property from typing import TYPE_CHECKING, Any -from airflow.hooks.base import BaseHook from pinecone import Pinecone, PodSpec, ServerlessSpec +from airflow.hooks.base import BaseHook + if TYPE_CHECKING: - from airflow.models.connection import Connection from pinecone import Vector from pinecone.core.client.model.sparse_values import SparseValues from pinecone.core.client.models import DescribeIndexStatsResponse, QueryResponse, UpsertResponse + from airflow.models.connection import Connection + class PineconeHook(BaseHook): """ diff --git a/providers/postgres/src/airflow/__init__.py b/providers/postgres/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/postgres/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/postgres/src/airflow/providers/__init__.py b/providers/postgres/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/postgres/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/presto/src/airflow/__init__.py b/providers/presto/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/presto/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/presto/src/airflow/providers/__init__.py b/providers/presto/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/presto/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/qdrant/src/airflow/__init__.py b/providers/qdrant/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/qdrant/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/qdrant/src/airflow/providers/__init__.py b/providers/qdrant/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/qdrant/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/redis/src/airflow/__init__.py b/providers/redis/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/redis/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/redis/src/airflow/providers/__init__.py b/providers/redis/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/redis/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/redis/src/airflow/providers/redis/hooks/redis.py b/providers/redis/src/airflow/providers/redis/hooks/redis.py index 35cb6ad83e742..2f956a5cb137e 100644 --- a/providers/redis/src/airflow/providers/redis/hooks/redis.py +++ b/providers/redis/src/airflow/providers/redis/hooks/redis.py @@ -21,9 +21,10 @@ from typing import Any -from airflow.hooks.base import BaseHook from redis import Redis +from airflow.hooks.base import BaseHook + DEFAULT_SSL_CERT_REQS = "required" ALLOWED_SSL_CERT_REQS = [DEFAULT_SSL_CERT_REQS, "optional", "none"] diff --git a/providers/redis/src/airflow/providers/redis/log/redis_task_handler.py b/providers/redis/src/airflow/providers/redis/log/redis_task_handler.py index 8f4f7a2adbf84..7107b2ab3a421 100644 --- a/providers/redis/src/airflow/providers/redis/log/redis_task_handler.py +++ b/providers/redis/src/airflow/providers/redis/log/redis_task_handler.py @@ -27,9 +27,10 @@ from airflow.utils.log.logging_mixin import LoggingMixin if TYPE_CHECKING: - from airflow.models import TaskInstance from redis import Redis + from airflow.models import TaskInstance + class RedisTaskHandler(FileTaskHandler, LoggingMixin): """ diff --git a/providers/salesforce/src/airflow/__init__.py b/providers/salesforce/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/salesforce/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/salesforce/src/airflow/providers/__init__.py b/providers/salesforce/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/salesforce/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/samba/src/airflow/__init__.py b/providers/samba/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/samba/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/samba/src/airflow/providers/__init__.py b/providers/samba/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/samba/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/segment/src/airflow/__init__.py b/providers/segment/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/segment/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/segment/src/airflow/providers/__init__.py b/providers/segment/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/segment/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/sendgrid/src/airflow/__init__.py b/providers/sendgrid/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/sendgrid/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/sendgrid/src/airflow/providers/__init__.py b/providers/sendgrid/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/sendgrid/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/sendgrid/src/airflow/providers/sendgrid/utils/emailer.py b/providers/sendgrid/src/airflow/providers/sendgrid/utils/emailer.py index 7a637e868ed51..f22a080deba78 100644 --- a/providers/sendgrid/src/airflow/providers/sendgrid/utils/emailer.py +++ b/providers/sendgrid/src/airflow/providers/sendgrid/utils/emailer.py @@ -27,8 +27,6 @@ from typing import Union import sendgrid -from airflow.hooks.base import BaseHook -from airflow.utils.email import get_email_address_list from sendgrid.helpers.mail import ( Attachment, Category, @@ -41,6 +39,9 @@ SandBoxMode, ) +from airflow.hooks.base import BaseHook +from airflow.utils.email import get_email_address_list + log = logging.getLogger(__name__) AddressesType = Union[str, Iterable[str]] diff --git a/providers/sftp/src/airflow/__init__.py b/providers/sftp/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/sftp/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/sftp/src/airflow/providers/__init__.py b/providers/sftp/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/sftp/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/singularity/src/airflow/__init__.py b/providers/singularity/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/singularity/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/singularity/src/airflow/providers/__init__.py b/providers/singularity/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/singularity/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/slack/src/airflow/__init__.py b/providers/slack/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/slack/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/slack/src/airflow/providers/__init__.py b/providers/slack/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/slack/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/smtp/src/airflow/__init__.py b/providers/smtp/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/smtp/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/smtp/src/airflow/providers/__init__.py b/providers/smtp/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/smtp/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/snowflake/src/airflow/__init__.py b/providers/snowflake/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/snowflake/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/snowflake/src/airflow/providers/__init__.py b/providers/snowflake/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/snowflake/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/snowflake/src/airflow/providers/snowflake/hooks/snowflake.py b/providers/snowflake/src/airflow/providers/snowflake/hooks/snowflake.py index 5777968b8d87c..f997d20aecdef 100644 --- a/providers/snowflake/src/airflow/providers/snowflake/hooks/snowflake.py +++ b/providers/snowflake/src/airflow/providers/snowflake/hooks/snowflake.py @@ -28,15 +28,15 @@ from cryptography.hazmat.backends import default_backend from cryptography.hazmat.primitives import serialization +from snowflake import connector +from snowflake.connector import DictCursor, SnowflakeConnection, util_text +from snowflake.sqlalchemy import URL from sqlalchemy import create_engine from airflow.exceptions import AirflowException from airflow.providers.common.sql.hooks.sql import DbApiHook, return_single_query_results from airflow.providers.snowflake.utils.openlineage import fix_snowflake_sqlalchemy_uri from airflow.utils.strings import to_boolean -from snowflake import connector -from snowflake.connector import DictCursor, SnowflakeConnection, util_text -from snowflake.sqlalchemy import URL T = TypeVar("T") if TYPE_CHECKING: @@ -367,9 +367,10 @@ def get_snowpark_session(self): :return: the created session. """ + from snowflake.snowpark import Session + from airflow import __version__ as airflow_version from airflow.providers.snowflake import __version__ as provider_version - from snowflake.snowpark import Session conn_config = self._get_conn_params session = Session.builder.configs(conn_config).create() diff --git a/providers/sqlite/src/airflow/__init__.py b/providers/sqlite/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/sqlite/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/sqlite/src/airflow/providers/__init__.py b/providers/sqlite/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/sqlite/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/ssh/src/airflow/__init__.py b/providers/ssh/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/ssh/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/ssh/src/airflow/providers/__init__.py b/providers/ssh/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/ssh/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/standard/src/airflow/__init__.py b/providers/standard/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/standard/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/standard/src/airflow/providers/__init__.py b/providers/standard/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/standard/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/tableau/src/airflow/__init__.py b/providers/tableau/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/tableau/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/tableau/src/airflow/providers/__init__.py b/providers/tableau/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/tableau/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/telegram/src/airflow/__init__.py b/providers/telegram/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/telegram/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/telegram/src/airflow/providers/__init__.py b/providers/telegram/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/telegram/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/telegram/src/airflow/providers/telegram/hooks/telegram.py b/providers/telegram/src/airflow/providers/telegram/hooks/telegram.py index 2f51a66766358..fd830c2250b69 100644 --- a/providers/telegram/src/airflow/providers/telegram/hooks/telegram.py +++ b/providers/telegram/src/airflow/providers/telegram/hooks/telegram.py @@ -22,9 +22,9 @@ import asyncio from typing import Any +import telegram import tenacity -import telegram from airflow.exceptions import AirflowException from airflow.hooks.base import BaseHook diff --git a/providers/teradata/src/airflow/__init__.py b/providers/teradata/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/teradata/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/teradata/src/airflow/providers/__init__.py b/providers/teradata/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/teradata/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/trino/src/airflow/__init__.py b/providers/trino/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/trino/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/trino/src/airflow/providers/__init__.py b/providers/trino/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/trino/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/trino/src/airflow/providers/trino/hooks/trino.py b/providers/trino/src/airflow/providers/trino/hooks/trino.py index c1a5f25bae354..af1e1b4264b20 100644 --- a/providers/trino/src/airflow/providers/trino/hooks/trino.py +++ b/providers/trino/src/airflow/providers/trino/hooks/trino.py @@ -24,14 +24,15 @@ from typing import TYPE_CHECKING, Any, TypeVar import trino +from trino.exceptions import DatabaseError +from trino.transaction import IsolationLevel + from airflow.configuration import conf from airflow.exceptions import AirflowException from airflow.providers.common.sql.hooks.sql import DbApiHook from airflow.providers.trino.version_compat import AIRFLOW_V_3_0_PLUS from airflow.utils.helpers import exactly_one from airflow.utils.operator_helpers import AIRFLOW_VAR_NAME_FORMAT_MAPPING, DEFAULT_FORMAT_PREFIX -from trino.exceptions import DatabaseError -from trino.transaction import IsolationLevel if TYPE_CHECKING: from airflow.models import Connection diff --git a/providers/vertica/src/airflow/__init__.py b/providers/vertica/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/vertica/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/vertica/src/airflow/providers/__init__.py b/providers/vertica/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/vertica/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/weaviate/src/airflow/__init__.py b/providers/weaviate/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/weaviate/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/weaviate/src/airflow/providers/__init__.py b/providers/weaviate/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/weaviate/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/weaviate/src/airflow/providers/weaviate/hooks/weaviate.py b/providers/weaviate/src/airflow/providers/weaviate/hooks/weaviate.py index 7287122f0cd01..e49cc58f02420 100644 --- a/providers/weaviate/src/airflow/providers/weaviate/hooks/weaviate.py +++ b/providers/weaviate/src/airflow/providers/weaviate/hooks/weaviate.py @@ -24,23 +24,21 @@ from typing import TYPE_CHECKING, Any, cast import requests -from tenacity import Retrying, retry, retry_if_exception, retry_if_exception_type, stop_after_attempt - import weaviate import weaviate.exceptions -from airflow.hooks.base import BaseHook +from tenacity import Retrying, retry, retry_if_exception, retry_if_exception_type, stop_after_attempt from weaviate import WeaviateClient from weaviate.auth import Auth from weaviate.classes.query import Filter from weaviate.exceptions import ObjectAlreadyExistsException from weaviate.util import generate_uuid5 +from airflow.hooks.base import BaseHook + if TYPE_CHECKING: from typing import Callable, Literal import pandas as pd - - from airflow.models.connection import Connection from weaviate.auth import AuthCredentials from weaviate.collections import Collection from weaviate.collections.classes.config import CollectionConfig, CollectionConfigSimple @@ -53,6 +51,8 @@ from weaviate.collections.classes.types import Properties from weaviate.types import UUID + from airflow.models.connection import Connection + ExitingSchemaOptions = Literal["replace", "fail", "ignore"] HTTP_RETRY_STATUS_CODE = [429, 500, 503, 504] diff --git a/providers/weaviate/src/airflow/providers/weaviate/operators/weaviate.py b/providers/weaviate/src/airflow/providers/weaviate/operators/weaviate.py index d6e133ceb52f9..1facdf77beb0d 100644 --- a/providers/weaviate/src/airflow/providers/weaviate/operators/weaviate.py +++ b/providers/weaviate/src/airflow/providers/weaviate/operators/weaviate.py @@ -26,7 +26,6 @@ if TYPE_CHECKING: import pandas as pd - from weaviate.types import UUID try: diff --git a/providers/yandex/src/airflow/__init__.py b/providers/yandex/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/yandex/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/yandex/src/airflow/providers/__init__.py b/providers/yandex/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/yandex/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/yandex/src/airflow/providers/yandex/secrets/lockbox.py b/providers/yandex/src/airflow/providers/yandex/secrets/lockbox.py index 0381486bab76e..d65131ab2cb1a 100644 --- a/providers/yandex/src/airflow/providers/yandex/secrets/lockbox.py +++ b/providers/yandex/src/airflow/providers/yandex/secrets/lockbox.py @@ -21,14 +21,14 @@ from functools import cached_property from typing import Any -import yandexcloud - import yandex.cloud.lockbox.v1.payload_pb2 as payload_pb import yandex.cloud.lockbox.v1.payload_service_pb2 as payload_service_pb import yandex.cloud.lockbox.v1.payload_service_pb2_grpc as payload_service_pb_grpc import yandex.cloud.lockbox.v1.secret_pb2 as secret_pb import yandex.cloud.lockbox.v1.secret_service_pb2 as secret_service_pb import yandex.cloud.lockbox.v1.secret_service_pb2_grpc as secret_service_pb_grpc +import yandexcloud + from airflow.models import Connection from airflow.providers.yandex.utils.credentials import get_credentials from airflow.providers.yandex.utils.defaults import default_conn_name diff --git a/providers/ydb/src/airflow/__init__.py b/providers/ydb/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/ydb/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/ydb/src/airflow/providers/__init__.py b/providers/ydb/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/ydb/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/ydb/src/airflow/providers/ydb/hooks/ydb.py b/providers/ydb/src/airflow/providers/ydb/hooks/ydb.py index d26d01699f0b4..2b1a319d3333f 100644 --- a/providers/ydb/src/airflow/providers/ydb/hooks/ydb.py +++ b/providers/ydb/src/airflow/providers/ydb/hooks/ydb.py @@ -19,10 +19,10 @@ from collections.abc import Mapping, Sequence from typing import TYPE_CHECKING, Any +import ydb from sqlalchemy.engine import URL from ydb_dbapi import Connection as DbApiConnection -import ydb from airflow.exceptions import AirflowException from airflow.providers.common.sql.hooks.sql import DbApiHook from airflow.providers.ydb.utils.credentials import get_credentials_from_connection diff --git a/providers/zendesk/src/airflow/__init__.py b/providers/zendesk/src/airflow/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/zendesk/src/airflow/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/providers/zendesk/src/airflow/providers/__init__.py b/providers/zendesk/src/airflow/providers/__init__.py new file mode 100644 index 0000000000000..e8fd22856438c --- /dev/null +++ b/providers/zendesk/src/airflow/providers/__init__.py @@ -0,0 +1,17 @@ +# 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. +__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore diff --git a/pyproject.toml b/pyproject.toml index 010b43813e471..bc57fedc851b6 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -330,6 +330,16 @@ testing = ["dev", "providers.tests", "task_sdk.tests", "tests_common", "tests"] # __path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore "providers/*/tests/*/__init__.py" = ["I002"] "providers/*/*/tests/*/*/__init__.py" = ["I002"] +"providers/*/src/airflow/__init__.py" = ["I002"] +"providers/*/src/airflow/providers/*/__init__.py" = ["I002"] +"providers/*/src/airflow/providers/__init__.py" = ["I002"] +"providers/*/*/src/airflow/providers/__init__.py" = ["I002"] +"providers/apache/*/src/airflow/providers/apache/__init__.py" = ["I002"] +"providers/atlassian/*/src/airflow/providers/atlassian/__init__.py" = ["I002"] +"providers/common/*/src/airflow/providers/common/__init__.py" = ["I002"] +"providers/cncf/*/src/airflow/providers/cncf/__init__.py" = ["I002"] +"providers/dbt/*/src/airflow/providers/dbt/__init__.py" = ["I002"] +"providers/microsoft/*/src/airflow/providers/microsoft/__init__.py" = ["I002"] # The test_python.py is needed because adding __future__.annotations breaks runtime checks that are # needed for the test to work @@ -593,17 +603,10 @@ show_error_codes = true disable_error_code = [ "annotation-unchecked", ] -# Since there are no __init__.py files in -# providers/src/apache/airflow/providers we need to tell MyPy where the "base" -# is, otherwise when it sees -# TODO(potiuk): check if those providers/*/src are still needed explicit_package_bases = true mypy_path = [ "$MYPY_CONFIG_FILE_DIR", "$MYPY_CONFIG_FILE_DIR/task_sdk/src", - "$MYPY_CONFIG_FILE_DIR/providers/airbyte/src", - "$MYPY_CONFIG_FILE_DIR/providers/celery/src", - "$MYPY_CONFIG_FILE_DIR/providers/edge/src", ] [[tool.mypy.overrides]] diff --git a/scripts/ci/pre_commit/check_providers_subpackages_all_have_init.py b/scripts/ci/pre_commit/check_providers_subpackages_all_have_init.py index 1c07570d19e8e..7139170aff723 100755 --- a/scripts/ci/pre_commit/check_providers_subpackages_all_have_init.py +++ b/scripts/ci/pre_commit/check_providers_subpackages_all_have_init.py @@ -27,9 +27,12 @@ ACCEPTED_NON_INIT_DIRS = [ "adr", "doc", + "3rd-party-licenses", "templates", "__pycache__", "static", + "dist", + "node_modules", ] PATH_EXTENSION_STRING = '__path__ = __import__("pkgutil").extend_path(__path__, __name__) # type: ignore' @@ -73,6 +76,21 @@ def _what_kind_of_test_init_py_needed(base_path: Path, folder: Path) -> tuple[bo return True, False +def _determine_init_py_action(need_path_extension: bool, root_path: Path): + init_py_file = root_path.joinpath("__init__.py") + if not init_py_file.exists(): + missing_init_dirs.append(root_path) + console.print(f"Missing __init__.py file {init_py_file}") + if need_path_extension: + missing_path_extension_dirs.append(root_path) + console.print(f"Missing path extension in: {init_py_file}") + elif need_path_extension: + text = init_py_file.read_text() + if PATH_EXTENSION_STRING not in text: + missing_path_extension_dirs.append(root_path) + console.print(f"Missing path extension in existing {init_py_file}") + + def check_dir_init_test_folders(folders: list[Path]) -> None: global fail_pre_commit folders = list(folders) @@ -83,37 +101,33 @@ def check_dir_init_test_folders(folders: list[Path]) -> None: for root, dirs, _ in os.walk(tests_folder): # Edit it in place, so we don't recurse to folders we don't care about dirs[:] = [d for d in dirs if d not in ACCEPTED_NON_INIT_DIRS] - need_init_py, need_path_extension = _what_kind_of_test_init_py_needed(tests_folder, Path(root)) + root_path = Path(root) + need_init_py, need_path_extension = _what_kind_of_test_init_py_needed(tests_folder, root_path) if need_init_py: - init_py_file = Path(root).joinpath("__init__.py") - if not init_py_file.exists(): - missing_init_dirs.append(Path(root)) - console.print(f"Missing __init__.py file {init_py_file}") - if need_path_extension: - missing_path_extension_dirs.append(Path(root)) - console.print(f"Missing path extension in: {init_py_file}") - elif need_path_extension: - text = init_py_file.read_text() - if PATH_EXTENSION_STRING not in text: - missing_path_extension_dirs.append(Path(root)) - console.print(f"Missing path extension in existing {init_py_file}") + _determine_init_py_action(need_path_extension, root_path) def check_dir_init_src_folders(folders: list[Path]) -> None: global fail_pre_commit folders = list(folders) for root_distribution_path in folders: - distribution_relative_path = root_distribution_path.relative_to(AIRFLOW_PROVIDERS_ROOT_PATH) # We need init folders for all folders and for the common ones we need path extension - provider_source_folder = root_distribution_path / "src" / distribution_relative_path - print("Checking for __init__.py files in distribution for src: ", provider_source_folder) - for root, dirs, _ in os.walk(provider_source_folder): + providers_base_folder = root_distribution_path / "src" / "airflow" + print("Checking for __init__.py files in distribution for src: ", providers_base_folder) + for root, dirs, _ in os.walk(providers_base_folder): + print("Checking: ", root) + root_path = Path(root) # Edit it in place, so we don't recurse to folders we don't care about dirs[:] = [d for d in dirs if d not in ACCEPTED_NON_INIT_DIRS] - init_py_file = Path(root).joinpath("__init__.py") - if not init_py_file.exists(): - missing_init_dirs.append(Path(root)) - console.print(f"Missing __init__.py file {init_py_file}") + relative_root_path = root_path.relative_to(providers_base_folder) + need_path_extension = ( + root_path == providers_base_folder + or len(relative_root_path.parts) == 2 + and relative_root_path.parts[0] in KNOWN_SECOND_LEVEL_PATHS + and relative_root_path.parts[1] == "providers" + ) + print("Needs path extension: ", need_path_extension) + _determine_init_py_action(need_path_extension, root_path) if __name__ == "__main__": diff --git a/scripts/ci/pre_commit/common_precommit_utils.py b/scripts/ci/pre_commit/common_precommit_utils.py index 627807c415286..e47b862775ba8 100644 --- a/scripts/ci/pre_commit/common_precommit_utils.py +++ b/scripts/ci/pre_commit/common_precommit_utils.py @@ -50,28 +50,31 @@ def read_airflow_version() -> str: def pre_process_files(files: list[str]) -> list[str]: """Pre-process files passed to mypy. + * Exclude conftest.py files and __init__.py files * When running build on non-main branch do not take providers into account. * When running "airflow/providers" package, then we need to add --namespace-packages flag. * When running "airflow" package, then we need to exclude providers. """ + + files = [file for file in files if not file.endswith("conftest.py") and not file.endswith("__init__.py")] default_branch = os.environ.get("DEFAULT_BRANCH") if not default_branch or default_branch == "main": return files - result = [file for file in files if not file.startswith(f"airflow{os.sep}providers")] + result = [file for file in files if not file.startswith("providers")] if "airflow/providers" in files: if len(files) > 1: raise RuntimeError( "When running `airflow/providers` package, you cannot run any other packages because only " "airflow/providers package requires --namespace-packages flag to be set" ) - result.append("--namespace-packages") + result.append("--no-namespace-packages") if "airflow" in files: if len(files) > 1: raise RuntimeError( "When running `airflow` package, you cannot run any other packages because only " - "airflow/providers package requires --exclude airflow/providers/.* flag to be set" + "airflow/providers package requires --exclude providers/.* flag to be set" ) - result.extend(["--exclude", "airflow/providers/.*"]) + result.extend(["--exclude", "providers/.*"]) return result diff --git a/scripts/ci/pre_commit/mypy.py b/scripts/ci/pre_commit/mypy.py index f57818342d70b..9904695de7338 100755 --- a/scripts/ci/pre_commit/mypy.py +++ b/scripts/ci/pre_commit/mypy.py @@ -33,7 +33,11 @@ initialize_breeze_precommit(__name__, __file__) files_to_test = pre_process_files(sys.argv[1:]) -if files_to_test == ["--namespace-packages"] or files_to_test == []: +if ( + files_to_test == ["--namespace-packages"] + or files_to_test == ["--no-namespace-packages"] + or files_to_test == [] +): print("No files to tests. Quitting") sys.exit(0) diff --git a/scripts/in_container/run_mypy.sh b/scripts/in_container/run_mypy.sh index 9552b7be4cdd9..12042efa50551 100755 --- a/scripts/in_container/run_mypy.sh +++ b/scripts/in_container/run_mypy.sh @@ -35,4 +35,5 @@ then ) done fi +set -x mypy "${ADDITIONAL_MYPY_OPTIONS[@]}" "${@}"