From 0c798252efb83844a2e0ba08d74b32eed2e317ad Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Tue, 25 Jul 2023 00:05:02 +0200 Subject: [PATCH] Extract Dask executor to new daskexecutor provider (#32772) --- .../airflow_providers_bug_report.yml | 1 + CONTRIBUTING.rst | 18 +- Dockerfile | 2 +- IMAGES.rst | 2 +- INSTALL | 18 +- airflow/config_templates/config.yml | 33 ---- airflow/executors/__init__.py | 3 + airflow/executors/executor_loader.py | 2 +- .../CREATING_COMMUNITY_PROVIDERS.rst | 74 ++++--- airflow/providers/daskexecutor/CHANGELOG.rst | 33 ++++ airflow/providers/daskexecutor/__init__.py | 41 ++++ .../daskexecutor/executors/__init__.py | 16 ++ .../daskexecutor}/executors/dask_executor.py | 0 airflow/providers/daskexecutor/provider.yaml | 79 ++++++++ .../src/airflow_breeze/global_constants.py | 2 +- .../changelog.rst | 19 ++ .../commits.rst | 27 +++ .../configurations-ref.rst | 18 ++ .../index.rst | 86 ++++++++ .../installing-providers-from-sources.rst | 18 ++ .../security.rst | 38 ++++ .../core-concepts/executor/dask.rst | 4 +- docs/apache-airflow/extra-packages-ref.rst | 10 +- docs/docker-stack/build-arg-ref.rst | 2 +- docs/integration-logos/dask/dask.png | Bin 0 -> 6134 bytes docs/spelling_wordlist.txt | 1 + generated/provider_dependencies.json | 10 + images/breeze/output-commands-hash.txt | 20 +- images/breeze/output_build-docs.svg | 128 ++++++------ images/breeze/output_prod-image.svg | 24 +-- images/breeze/output_prod-image_build.svg | 184 +++++++++--------- images/breeze/output_release-management.svg | 171 ++++++++++++++++ ...ement_generate-issue-content-providers.svg | 72 +++---- ...agement_prepare-provider-documentation.svg | 66 +++---- ...e-management_prepare-provider-packages.svg | 40 ++-- ...output_release-management_publish-docs.svg | 110 +++++------ images/breeze/output_sbom.svg | 20 +- ...ut_sbom_generate-provider-requirements.svg | 54 ++--- scripts/ci/installed_providers.txt | 1 + .../ci/pre_commit/pre_commit_insert_extras.py | 2 +- setup.py | 9 +- tests/providers/daskexecutor/__init__.py | 16 ++ .../daskexecutor}/test_dask_executor.py | 4 +- 43 files changed, 1028 insertions(+), 450 deletions(-) create mode 100644 airflow/providers/daskexecutor/CHANGELOG.rst create mode 100644 airflow/providers/daskexecutor/__init__.py create mode 100644 airflow/providers/daskexecutor/executors/__init__.py rename airflow/{ => providers/daskexecutor}/executors/dask_executor.py (100%) create mode 100644 airflow/providers/daskexecutor/provider.yaml create mode 100644 docs/apache-airflow-providers-daskexecutor/changelog.rst create mode 100644 docs/apache-airflow-providers-daskexecutor/commits.rst create mode 100644 docs/apache-airflow-providers-daskexecutor/configurations-ref.rst create mode 100644 docs/apache-airflow-providers-daskexecutor/index.rst create mode 100644 docs/apache-airflow-providers-daskexecutor/installing-providers-from-sources.rst create mode 100644 docs/apache-airflow-providers-daskexecutor/security.rst create mode 100644 docs/integration-logos/dask/dask.png create mode 100644 images/breeze/output_release-management.svg create mode 100644 tests/providers/daskexecutor/__init__.py rename tests/{executors => providers/daskexecutor}/test_dask_executor.py (98%) diff --git a/.github/ISSUE_TEMPLATE/airflow_providers_bug_report.yml b/.github/ISSUE_TEMPLATE/airflow_providers_bug_report.yml index cb50fcb0c74b0..02f6b38a852c5 100644 --- a/.github/ISSUE_TEMPLATE/airflow_providers_bug_report.yml +++ b/.github/ISSUE_TEMPLATE/airflow_providers_bug_report.yml @@ -49,6 +49,7 @@ body: - cloudant - cncf-kubernetes - common-sql + - daskexecutor - databricks - datadog - dbt-cloud diff --git a/CONTRIBUTING.rst b/CONTRIBUTING.rst index 06b9391eb9341..ef358cbdc107c 100644 --- a/CONTRIBUTING.rst +++ b/CONTRIBUTING.rst @@ -671,15 +671,15 @@ aiobotocore, airbyte, alibaba, all, all_dbs, amazon, apache.atlas, apache.beam, apache.drill, apache.druid, apache.flink, apache.hdfs, apache.hive, apache.impala, apache.kafka, apache.kylin, apache.livy, apache.pig, apache.pinot, apache.spark, apache.sqoop, apache.webhdfs, apprise, arangodb, asana, async, atlas, atlassian.jira, aws, azure, cassandra, celery, cgroups, -cloudant, cncf.kubernetes, common.sql, crypto, dask, databricks, datadog, dbt.cloud, deprecated_api, -devel, devel_all, devel_ci, devel_hadoop, dingding, discord, doc, doc_gen, docker, druid, -elasticsearch, exasol, facebook, ftp, gcp, gcp_api, github, github_enterprise, google, google_auth, -grpc, hashicorp, hdfs, hive, http, imap, influxdb, jdbc, jenkins, kerberos, kubernetes, ldap, -leveldb, microsoft.azure, microsoft.mssql, microsoft.psrp, microsoft.winrm, mongo, mssql, mysql, -neo4j, odbc, openfaas, openlineage, opsgenie, oracle, otel, pagerduty, pandas, papermill, password, -pinot, plexus, postgres, presto, qds, qubole, rabbitmq, redis, s3, salesforce, samba, segment, -sendgrid, sentry, sftp, singularity, slack, smtp, snowflake, spark, sqlite, ssh, statsd, tableau, -tabular, telegram, trino, vertica, virtualenv, webhdfs, winrm, zendesk +cloudant, cncf.kubernetes, common.sql, crypto, dask, daskexecutor, databricks, datadog, dbt.cloud, +deprecated_api, devel, devel_all, devel_ci, devel_hadoop, dingding, discord, doc, doc_gen, docker, +druid, elasticsearch, exasol, facebook, ftp, gcp, gcp_api, github, github_enterprise, google, +google_auth, grpc, hashicorp, hdfs, hive, http, imap, influxdb, jdbc, jenkins, kerberos, kubernetes, +ldap, leveldb, microsoft.azure, microsoft.mssql, microsoft.psrp, microsoft.winrm, mongo, mssql, +mysql, neo4j, odbc, openfaas, openlineage, opsgenie, oracle, otel, pagerduty, pandas, papermill, +password, pinot, plexus, postgres, presto, qds, qubole, rabbitmq, redis, s3, salesforce, samba, +segment, sendgrid, sentry, sftp, singularity, slack, smtp, snowflake, spark, sqlite, ssh, statsd, +tableau, tabular, telegram, trino, vertica, virtualenv, webhdfs, winrm, zendesk .. END EXTRAS HERE Provider packages diff --git a/Dockerfile b/Dockerfile index 15ee45f7e3bf7..16446c0b03e70 100644 --- a/Dockerfile +++ b/Dockerfile @@ -35,7 +35,7 @@ # much smaller. # # Use the same builder frontend version for everyone -ARG AIRFLOW_EXTRAS="aiobotocore,amazon,async,celery,cncf.kubernetes,dask,docker,elasticsearch,ftp,google,google_auth,grpc,hashicorp,http,ldap,microsoft.azure,mysql,odbc,pandas,postgres,redis,sendgrid,sftp,slack,snowflake,ssh,statsd,virtualenv" +ARG AIRFLOW_EXTRAS="aiobotocore,amazon,async,celery,cncf.kubernetes,daskexecutor,docker,elasticsearch,ftp,google,google_auth,grpc,hashicorp,http,ldap,microsoft.azure,mysql,odbc,pandas,postgres,redis,sendgrid,sftp,slack,snowflake,ssh,statsd,virtualenv" ARG ADDITIONAL_AIRFLOW_EXTRAS="" ARG ADDITIONAL_PYTHON_DEPS="" diff --git a/IMAGES.rst b/IMAGES.rst index 7360e43648f44..9c34fce80d6ea 100644 --- a/IMAGES.rst +++ b/IMAGES.rst @@ -135,7 +135,7 @@ This will build the image using command similar to: .. code-block:: bash pip install \ - apache-airflow[async,amazon,celery,cncf.kubernetes,docker,dask,elasticsearch,ftp,grpc,hashicorp,http,ldap,google,microsoft.azure,mysql,postgres,redis,sendgrid,sftp,slack,ssh,statsd,virtualenv]==2.0.0 \ + apache-airflow[async,amazon,celery,cncf.kubernetes,docker,daskexecutor,elasticsearch,ftp,grpc,hashicorp,http,ldap,google,microsoft.azure,mysql,postgres,redis,sendgrid,sftp,slack,ssh,statsd,virtualenv]==2.0.0 \ --constraint "https://raw.githubusercontent.com/apache/airflow/constraints-2.0.0/constraints-3.8.txt" .. note:: diff --git a/INSTALL b/INSTALL index c5bfc7fe55d03..b30f0dbd5f10f 100644 --- a/INSTALL +++ b/INSTALL @@ -98,15 +98,15 @@ aiobotocore, airbyte, alibaba, all, all_dbs, amazon, apache.atlas, apache.beam, apache.drill, apache.druid, apache.flink, apache.hdfs, apache.hive, apache.impala, apache.kafka, apache.kylin, apache.livy, apache.pig, apache.pinot, apache.spark, apache.sqoop, apache.webhdfs, apprise, arangodb, asana, async, atlas, atlassian.jira, aws, azure, cassandra, celery, cgroups, -cloudant, cncf.kubernetes, common.sql, crypto, dask, databricks, datadog, dbt.cloud, deprecated_api, -devel, devel_all, devel_ci, devel_hadoop, dingding, discord, doc, doc_gen, docker, druid, -elasticsearch, exasol, facebook, ftp, gcp, gcp_api, github, github_enterprise, google, google_auth, -grpc, hashicorp, hdfs, hive, http, imap, influxdb, jdbc, jenkins, kerberos, kubernetes, ldap, -leveldb, microsoft.azure, microsoft.mssql, microsoft.psrp, microsoft.winrm, mongo, mssql, mysql, -neo4j, odbc, openfaas, openlineage, opsgenie, oracle, otel, pagerduty, pandas, papermill, password, -pinot, plexus, postgres, presto, qds, qubole, rabbitmq, redis, s3, salesforce, samba, segment, -sendgrid, sentry, sftp, singularity, slack, smtp, snowflake, spark, sqlite, ssh, statsd, tableau, -tabular, telegram, trino, vertica, virtualenv, webhdfs, winrm, zendesk +cloudant, cncf.kubernetes, common.sql, crypto, dask, daskexecutor, databricks, datadog, dbt.cloud, +deprecated_api, devel, devel_all, devel_ci, devel_hadoop, dingding, discord, doc, doc_gen, docker, +druid, elasticsearch, exasol, facebook, ftp, gcp, gcp_api, github, github_enterprise, google, +google_auth, grpc, hashicorp, hdfs, hive, http, imap, influxdb, jdbc, jenkins, kerberos, kubernetes, +ldap, leveldb, microsoft.azure, microsoft.mssql, microsoft.psrp, microsoft.winrm, mongo, mssql, +mysql, neo4j, odbc, openfaas, openlineage, opsgenie, oracle, otel, pagerduty, pandas, papermill, +password, pinot, plexus, postgres, presto, qds, qubole, rabbitmq, redis, s3, salesforce, samba, +segment, sendgrid, sentry, sftp, singularity, slack, smtp, snowflake, spark, sqlite, ssh, statsd, +tableau, tabular, telegram, trino, vertica, virtualenv, webhdfs, winrm, zendesk # END EXTRAS HERE # For installing Airflow in development environments - see CONTRIBUTING.rst diff --git a/airflow/config_templates/config.yml b/airflow/config_templates/config.yml index 47f21ee7369bf..35f16c57a8f43 100644 --- a/airflow/config_templates/config.yml +++ b/airflow/config_templates/config.yml @@ -1958,39 +1958,6 @@ local_kubernetes_executor: type: string example: ~ default: "kubernetes" -dask: - description: | - This section only applies if you are using the DaskExecutor in - [core] section above - options: - cluster_address: - description: | - The IP address and port of the Dask cluster's scheduler. - version_added: ~ - type: string - example: ~ - default: "127.0.0.1:8786" - tls_ca: - description: | - Path to a CA certificate file encoded in PEM format to access a secured Dask scheduler. - version_added: ~ - type: string - example: ~ - default: "" - tls_cert: - description: | - Path to a certificate file for the client, encoded in PEM format. - version_added: ~ - type: string - example: ~ - default: "" - tls_key: - description: | - Path to a key file for the client, encoded in PEM format. - version_added: ~ - type: string - example: ~ - default: "" scheduler: description: ~ options: diff --git a/airflow/executors/__init__.py b/airflow/executors/__init__.py index 2c342d45f16d9..ceeec523397b5 100644 --- a/airflow/executors/__init__.py +++ b/airflow/executors/__init__.py @@ -28,6 +28,9 @@ "CeleryKubernetesExecutor": "airflow.providers.celery.executors." "celery_kubernetes_executor.CeleryKubernetesExecutor", }, + "dask_executor": { + "DaskExecutor": "airflow.providers.daskexecutor.executors.dask_executor.DaskExecutor", + }, } add_deprecated_classes(__deprecated_classes, __name__) diff --git a/airflow/executors/executor_loader.py b/airflow/executors/executor_loader.py index 816e172140aee..3553a71183609 100644 --- a/airflow/executors/executor_loader.py +++ b/airflow/executors/executor_loader.py @@ -63,7 +63,7 @@ class ExecutorLoader: CELERY_EXECUTOR: "airflow.providers.celery.executors.celery_executor.CeleryExecutor", CELERY_KUBERNETES_EXECUTOR: "airflow.providers.celery." "executors.celery_kubernetes_executor.CeleryKubernetesExecutor", - DASK_EXECUTOR: "airflow.executors.dask_executor.DaskExecutor", + DASK_EXECUTOR: "airflow.providers.daskexecutor.executors.dask_executor.DaskExecutor", KUBERNETES_EXECUTOR: "airflow.executors.kubernetes_executor.KubernetesExecutor", DEBUG_EXECUTOR: "airflow.executors.debug_executor.DebugExecutor", } diff --git a/airflow/providers/CREATING_COMMUNITY_PROVIDERS.rst b/airflow/providers/CREATING_COMMUNITY_PROVIDERS.rst index be249b53bb046..d4e14e138cf66 100644 --- a/airflow/providers/CREATING_COMMUNITY_PROVIDERS.rst +++ b/airflow/providers/CREATING_COMMUNITY_PROVIDERS.rst @@ -55,6 +55,9 @@ the provider may need. Understand that not all providers will need all the compo If you still have doubts about building your provider, we recommend that you read the initial provider guide and open a issue on GitHub so the community can help you. +The folders are optional: example_dags, hooks, links, logs, notifications, operators, secrets, sensors, transfers, +triggers, waiters (and the list changes continuously). + .. code-block:: bash airflow/ @@ -63,20 +66,27 @@ open a issue on GitHub so the community can help you. │ ├── example_dags/ │ │ ├── __init__.py │ │ └── example_.py + │ ├── executors/ + │ │ ├── __init__.py + │ │ └── .py │ ├── hooks/ │ │ ├── __init__.py │ │ └── .py │ ├── operators/ │ │ ├── __init__.py │ │ └── .py - │ ├── sensors/ + .... + │ ├── transfers/ │ │ ├── __init__.py │ │ └── .py - │ └── transfers/ + │ └── triggers/ │ ├── __init__.py │ └── .py └── tests/providers// ├── __init__.py + ├── executors/ + │ ├── __init__.py + │ └── test_.py ├── hooks/ │ ├── __init__.py │ └── test_.py @@ -84,10 +94,11 @@ open a issue on GitHub so the community can help you. │ ├── __init__.py │ ├── test_.py │ └── test__system.py - ├── sensors/ + ... + ├── transfers/ │ ├── __init__.py │ └── test_.py - └── transfers/ + └── triggers/ ├── __init__.py └── test_.py @@ -113,26 +124,27 @@ Some steps for documentation occurs automatically by ``pre-commit`` see `Install .. code-block:: bash - airflow/ - ├── INSTALL - ├── CONTRIBUTING.rst - ├── setup.py - ├── docs/ - │ ├── spelling_wordlist.txt - │ ├── apache-airflow/ - │ │ └── extra-packages-ref.rst - │ ├── integration-logos// - │ │ └── .png - │ └── apache-airflow-providers-/ - │ ├── index.rst - │ ├── commits.rst - │ ├── connections.rst - │ └── operators/ - │ └── .rst - └── providers/ - └── / - ├── provider.yaml - └── CHANGELOG.rst + ├── INSTALL + ├── CONTRIBUTING.rst + ├── setup.py + ├── airflow/ + │ └── providers/ + │ └── / + │ ├── provider.yaml + │ └── CHANGELOG.rst + │ + └── docs/ + ├── spelling_wordlist.txt + ├── apache-airflow/ + │ └── extra-packages-ref.rst + ├── integration-logos// + │ └── .png + └── apache-airflow-providers-/ + ├── index.rst + ├── commits.rst + ├── connections.rst + └── operators/ + └── .rst Files automatically updated by pre-commit: @@ -199,9 +211,19 @@ In the ``docs/apache-airflow-providers-/operators/.r :end-before: [END howto_operator_] -In the ``docs/apache-airflow-providers-new_provider/index.rst``: +Copy from another, similar provider the docs: ``docs/apache-airflow-providers-new_provider/*.rst``: + +At least those docs should be present + +* security.rst +* changelog.rst +* commits.rst +* index.rst +* installing-providers-from-sources.rst +* configurations-ref.rst - if your provider has ``config`` element in provider.yaml with configuration options + specific for your provider -- add all information of the purpose of your provider. It is recommended to check with another provider to help you complete this document as best as possible. +Make sure to update/add all information that are specific for the new provider. In the ``airflow/providers//provider.yaml`` add information of your provider: diff --git a/airflow/providers/daskexecutor/CHANGELOG.rst b/airflow/providers/daskexecutor/CHANGELOG.rst new file mode 100644 index 0000000000000..264edd3a39665 --- /dev/null +++ b/airflow/providers/daskexecutor/CHANGELOG.rst @@ -0,0 +1,33 @@ + .. 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. + + +.. NOTE TO CONTRIBUTORS: + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes + and you want to add an explanation to the users on how they are supposed to deal with them. + The changelog is updated and maintained semi-automatically by release manager. + +``apache-airflow-providers-daskexecutor`` + + +Changelog +--------- + +1.0.0 +..... + +Initial version of the provider. diff --git a/airflow/providers/daskexecutor/__init__.py b/airflow/providers/daskexecutor/__init__.py new file mode 100644 index 0000000000000..6548d7b15d61d --- /dev/null +++ b/airflow/providers/daskexecutor/__init__.py @@ -0,0 +1,41 @@ +# +# 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. +# +# NOTE! THIS FILE IS AUTOMATICALLY GENERATED AND WILL BE +# OVERWRITTEN WHEN PREPARING DOCUMENTATION FOR THE PACKAGES. +# +# IF YOU WANT TO MODIFY IT, YOU SHOULD MODIFY THE TEMPLATE +# `PROVIDER__INIT__PY_TEMPLATE.py.jinja2` IN the `dev/provider_packages` DIRECTORY +# +from __future__ import annotations + +import packaging.version + +__all__ = ["__version__"] + +__version__ = "1.0.0" + +try: + from airflow import __version__ as airflow_version +except ImportError: + from airflow.version import version as airflow_version + +if packaging.version.parse(airflow_version) < packaging.version.parse("2.4.0"): + raise RuntimeError( + f"The package `apache-airflow-providers-daskexecutor:{__version__}` requires Apache Airflow 2.4.0+" # NOQA: E501 + ) diff --git a/airflow/providers/daskexecutor/executors/__init__.py b/airflow/providers/daskexecutor/executors/__init__.py new file mode 100644 index 0000000000000..13a83393a9124 --- /dev/null +++ b/airflow/providers/daskexecutor/executors/__init__.py @@ -0,0 +1,16 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. diff --git a/airflow/executors/dask_executor.py b/airflow/providers/daskexecutor/executors/dask_executor.py similarity index 100% rename from airflow/executors/dask_executor.py rename to airflow/providers/daskexecutor/executors/dask_executor.py diff --git a/airflow/providers/daskexecutor/provider.yaml b/airflow/providers/daskexecutor/provider.yaml new file mode 100644 index 0000000000000..3c6466d460403 --- /dev/null +++ b/airflow/providers/daskexecutor/provider.yaml @@ -0,0 +1,79 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +--- +package-name: apache-airflow-providers-daskexecutor +name: Dask Executor +description: | + `Dask `__ + +suspended: false +versions: + - 1.0.0 + +dependencies: + - apache-airflow>=2.4.0 + # Dask support is limited, we need Dask team to upgrade support for dask if we were to continue + # Supporting it in the future + - cloudpickle>=1.4.1 + # Dask and distributed in version 2023.5.0 break our test + # See https://github.com/dask/dask/issues/10279 + - dask>=2.9.0,!=2022.10.1,!=2023.5.0 + - distributed>=2.11.1,!=2023.5.0 + +integrations: + - integration-name: Dask + external-doc-url: https://www.dask.org/ + logo: /integration-logos/dask/dask.png + tags: [service] + +executors: + - airflow.providers.daskexecutor.executors.dask_executor.DaskExecutor + +config: + dask: + description: | + This section only applies if you are using DaskExecutor. + options: + cluster_address: + description: | + The IP address and port of the Dask cluster's scheduler. + version_added: ~ + type: string + example: ~ + default: "127.0.0.1:8786" + tls_ca: + description: | + Path to a CA certificate file encoded in PEM format to access a secured Dask scheduler. + version_added: ~ + type: string + example: ~ + default: "" + tls_cert: + description: | + Path to a certificate file for the client, encoded in PEM format. + version_added: ~ + type: string + example: ~ + default: "" + tls_key: + description: | + Path to a key file for the client, encoded in PEM format. + version_added: ~ + type: string + example: ~ + default: "" diff --git a/dev/breeze/src/airflow_breeze/global_constants.py b/dev/breeze/src/airflow_breeze/global_constants.py index 9cdca2edb3bb8..444bd1e01048e 100644 --- a/dev/breeze/src/airflow_breeze/global_constants.py +++ b/dev/breeze/src/airflow_breeze/global_constants.py @@ -341,7 +341,7 @@ def get_airflow_extras(): "async", "celery", "cncf.kubernetes", - "dask", + "daskexecutor", "docker", "elasticsearch", "ftp", diff --git a/docs/apache-airflow-providers-daskexecutor/changelog.rst b/docs/apache-airflow-providers-daskexecutor/changelog.rst new file mode 100644 index 0000000000000..a2bf898b3f6a3 --- /dev/null +++ b/docs/apache-airflow-providers-daskexecutor/changelog.rst @@ -0,0 +1,19 @@ + + .. 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. + +.. include:: ../../airflow/providers/daskexecutor/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-daskexecutor/commits.rst b/docs/apache-airflow-providers-daskexecutor/commits.rst new file mode 100644 index 0000000000000..1b8a85877d0cf --- /dev/null +++ b/docs/apache-airflow-providers-daskexecutor/commits.rst @@ -0,0 +1,27 @@ + + .. 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. + + +Package apache-airflow-providers-daskexecutor +------------------------------------------------------ + +`Dask `__ + + +This is detailed commit list of changes for versions provider package: ``daskexecutor``. +For high-level changelog, see :doc:`package information including changelog `. diff --git a/docs/apache-airflow-providers-daskexecutor/configurations-ref.rst b/docs/apache-airflow-providers-daskexecutor/configurations-ref.rst new file mode 100644 index 0000000000000..5885c9d91b6e8 --- /dev/null +++ b/docs/apache-airflow-providers-daskexecutor/configurations-ref.rst @@ -0,0 +1,18 @@ + .. 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. + +.. include:: ../exts/includes/providers-configurations-ref.rst diff --git a/docs/apache-airflow-providers-daskexecutor/index.rst b/docs/apache-airflow-providers-daskexecutor/index.rst new file mode 100644 index 0000000000000..be612a6956320 --- /dev/null +++ b/docs/apache-airflow-providers-daskexecutor/index.rst @@ -0,0 +1,86 @@ + + .. 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. + +``apache-airflow-providers-daskexecutor`` +========================================= + + +.. toctree:: + :hidden: + :maxdepth: 1 + :caption: Basics + + Home + Changelog + Security + +.. toctree:: + :hidden: + :maxdepth: 1 + :caption: References + + Configuration + Python API <_api/airflow/providers/daskexecutor/index> + PyPI Repository + Installing from sources + +.. THE REMAINDER OF THE FILE IS AUTOMATICALLY GENERATED. IT WILL BE OVERWRITTEN AT RELEASE TIME! + + +.. toctree:: + :hidden: + :maxdepth: 1 + :caption: Commits + + Detailed list of commits + + +Package apache-airflow-providers-daskexecutor +------------------------------------------------------ + +`Dask `__ + + +Release: 1.0.0 + +Provider package +---------------- + +This is a provider package for ``daskexecutor`` provider. All classes for this provider package +are in ``airflow.providers.daskexecutor`` python package. + +Installation +------------ + +You can install this package on top of an existing Airflow 2 installation (see ``Requirements`` below) +for the minimum Airflow version supported) via +``pip install apache-airflow-providers-daskexecutor`` + +Requirements +------------ + +The minimum Apache Airflow version supported by this provider package is ``2.4.0``. + +================== ================================== +PIP package Version required +================== ================================== +``apache-airflow`` ``>=2.4.0`` +``cloudpickle`` ``>=1.4.1`` +``dask`` ``>=2.9.0,!=2022.10.1,!=2023.5.0`` +``distributed`` ``>=2.11.1,!=2023.5.0`` +================== ================================== diff --git a/docs/apache-airflow-providers-daskexecutor/installing-providers-from-sources.rst b/docs/apache-airflow-providers-daskexecutor/installing-providers-from-sources.rst new file mode 100644 index 0000000000000..b4e730f4ff21a --- /dev/null +++ b/docs/apache-airflow-providers-daskexecutor/installing-providers-from-sources.rst @@ -0,0 +1,18 @@ + .. 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. + +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-daskexecutor/security.rst b/docs/apache-airflow-providers-daskexecutor/security.rst new file mode 100644 index 0000000000000..66c6f79a4ecfc --- /dev/null +++ b/docs/apache-airflow-providers-daskexecutor/security.rst @@ -0,0 +1,38 @@ + + .. 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. + +Releasing security patches +-------------------------- + +Airflow providers are released independently from Airflow itself and the information about vulnerabilities +is published separately. You can upgrade providers independently from Airflow itself, following the +instructions found in :doc:`apache-airflow:installation/installing-from-pypi`. + +When we release Provider version, the development is always done from the ``main`` branch where we prepare +the next version. The provider uses strict `SemVer `_ versioning policy. Depending on +the scope of the change, Provider will get ''MAJOR'' version upgrade when there are +breaking changes, ``MINOR`` version upgrade when there are new features or ``PATCHLEVEL`` version upgrade +when there are only bug fixes (including security bugfixes) - and this is the only version that receives +security fixes by default, so you should upgrade to latest version of the provider if you want to receive +all released security fixes. + +The only exception to that rule is when we have a critical security fix and good reason to provide an +out-of-band release for the provider, in which case stakeholders in the provider might decide to cherry-pick +and prepare a branch for an older version of the provider following the +`mixed governance model `_ +and requires interested parties to cherry-pick and test the fixes. diff --git a/docs/apache-airflow/core-concepts/executor/dask.rst b/docs/apache-airflow/core-concepts/executor/dask.rst index 7551ffaf8b38e..117d396962976 100644 --- a/docs/apache-airflow/core-concepts/executor/dask.rst +++ b/docs/apache-airflow/core-concepts/executor/dask.rst @@ -21,7 +21,7 @@ Dask Executor ============= -:class:`airflow.executors.dask_executor.DaskExecutor` allows you to run Airflow tasks in a Dask Distributed cluster. +:class:`airflow.providers.daskexecutor.executors.dask_executor.DaskExecutor` allows you to run Airflow tasks in a Dask Distributed cluster. Dask clusters can be run on a single machine or on remote networks. For complete details, consult the `Distributed documentation `_. @@ -42,7 +42,7 @@ Next start at least one Worker on any machine that can connect to the host: dask-worker $DASK_HOST:$DASK_PORT -Edit your ``airflow.cfg`` to set your executor to :class:`airflow.executors.dask_executor.DaskExecutor` and provide +Edit your ``airflow.cfg`` to set your executor to :class:`airflow.providers.daskexecutor.executors.dask_executor.DaskExecutor` and provide the Dask Scheduler address in the ``[dask]`` section. For more information on setting the configuration, see :doc:`../../howto/set-config`. diff --git a/docs/apache-airflow/extra-packages-ref.rst b/docs/apache-airflow/extra-packages-ref.rst index 027fbb657e415..ef6c4bf1b2124 100644 --- a/docs/apache-airflow/extra-packages-ref.rst +++ b/docs/apache-airflow/extra-packages-ref.rst @@ -50,7 +50,7 @@ python dependencies for the provided package. +---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+--------------+ | cncf.kubernetes | ``pip install 'apache-airflow[cncf.kubernetes]'`` | Kubernetes Executor (also installs the Kubernetes provider package) | | +---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+--------------+ -| dask | ``pip install 'apache-airflow[dask]'`` | DaskExecutor | | +| daskexecutor | ``pip install 'apache-airflow[daskexecutor]'`` | DaskExecutor ((also installs the Daskexecutor provider package) | | +---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+--------------+ | deprecated_api | ``pip install 'apache-airflow[deprecated_api]'`` | Deprecated, experimental API that is replaced with the new REST API | | +---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+--------------+ @@ -349,10 +349,10 @@ Those are the extras that are needed to generated documentation for Airflow. Thi +---------------------+-----------------------------------------------------+----------------------------------------------------------------------+ -Deprecated 1.10 extras ----------------------- +Deprecated extras +----------------- -These are the extras that have been deprecated in 2.0 and will be removed in Airflow 3.0.0. They were +These are the extras that have been used before and deprecated in 2.0 and will be removed in Airflow 3.0.0. They were all replaced by new extras, which have naming consistent with the names of provider packages. The ``crypto`` extra is not needed any more, because all crypto dependencies are part of airflow package, @@ -371,6 +371,8 @@ so there is no replacement for ``crypto`` extra. +---------------------+-----------------------------+ | crypto | | +---------------------+-----------------------------+ +| dask | daskexecutor | ++---------------------+-----------------------------+ | druid | apache.druid | +---------------------+-----------------------------+ | gcp | google | diff --git a/docs/docker-stack/build-arg-ref.rst b/docs/docker-stack/build-arg-ref.rst index c954cbf69c7e3..a142968b5b4fb 100644 --- a/docs/docker-stack/build-arg-ref.rst +++ b/docs/docker-stack/build-arg-ref.rst @@ -85,7 +85,7 @@ List of default extras in the production Dockerfile: * async * celery * cncf.kubernetes -* dask +* daskexecutor * docker * elasticsearch * ftp diff --git a/docs/integration-logos/dask/dask.png b/docs/integration-logos/dask/dask.png new file mode 100644 index 0000000000000000000000000000000000000000..1c2b27ee3d364ac288afff556c1cbe411c239eb4 GIT binary patch literal 6134 zcmaKQcQjmG)V}D9E^5>W2?o)L61|tG8KY&0FuGy#>UBnqmceKtdh{Vi36dB+qW4~h z;3bGCL45Q6zTe;9T4&vL*IE1SXP;+3_w2oP!ZQ0NJI}N=6lDi)eVEd12;RhVYPw#uW(L0 z-83Clw)I!Cq}-NjW~6;&^dIiPj2_1yf4yTC)x*d5=5Z>7`nPqdrQ^N+FGTX&#Q{JF(aopRVxAKKbNaMB{9%+KmNa#w9yCNM6YSujp5T# zj#x-6IlMDaDc+q$t2c*bpTA;!R+6Thh6es|j-@9+SZ^;=(FX3C3DQ_3IJHdt(=bOJ z^32aP(V$A6eyo{*lX<7J{0p&}if0;1vASobiQIw-dqlK8sJ)wTRx+H;-;QK3? zbP3t)O!NP6XN0E2oJ<)8@wuRH12vCR7|m*DJTmeXwT$DAUF?bF<&xx7``%d~=6&=EvhzBpI;{IdRZ; zzO3Spj48iZxIC-gY2H1kE#6KhFfZsCo-pI|pX0bo5nx(>IA8iAt&^}obrD5N>T9~> z!P%k!8Ah85Kh@en_7{+3x?jeiJ?jSMfxY{gP5&A;1d+l@U(=ybwjVHXVmsoxNx?0K zL&$s=j+1*k33F7yXqWPim^f&Lfc{JQ^ux&G+CCz5I7Ow~wyLqJ4Gz~BL!jt81tdz1 z4(Tv$N<)A~=fsMBc*GgUGd-OfaDqU(P!RZLcY&zWyRtD17{pk98?aXhH&lv4nXXt5 z&g`f^mhiw0=Doq0JGnm;+;TV_)n)zfVDB}_}jM`u;P z>mTEF-G1S0#<*e%chxR5I!yyDXfN7Kn3zujE^Z$-MOX(u?O5Y>N*!D+HKzDBVzHPg zc`2^Y-@V?Mu(rD2)Ap+mN#9xd)IZp<@o^*Wllph}x5DD9)-CaV2ie2hJbdZ(9!|od zVTiJx32rqVw&)DGy|;TQ*Fs)Ok6+t%(1u2lTzp-w2_u4DA8}k1+W~G>jNh1kL1yJT zkKs=6Kf8m%$+WH_Uw3?_ir~9ms~&R*S+JuDZ@N3OeqZ5l#NJL8)6;9Sr8=S8{H*W2 zt*2OL!Lmn?>GIdnh4{^A@h;EAr-D#_(Yq@a6WS<=j~l83?O131HY~O4$QZ)>b#WiX zkG23JvoO+^r}^R!Euss@&N5><6Tksdp^w8W!;UNvb<` zQcA81m{R+0-?C zt%dMh$X5x0u+5L;A#TeT>*%K02QK%o-AosOQ(H9g=M~_C;*B`de)LP1NOUQvkmqCk z#Yry+5%i%kPFA%|r6ZNz9d5x~SL#ypOq{2CZ`y@lRpm>a&Ncc*Ga{W~ zylD&*&8}3iDOhmOA`$>I+l*RZzuLL zxlKnVECxwuim*d8-$*4|>?#dF1guk_cFVB&kbSrlci8Rgqk>{YJeA!?iPIYr6S8HvRlXk0R z+Mgg2g6y;rb=U1IC@>MRq6GMuje()Z-E3O5$01E`m__yl*K1less;1nBGq^Z`gv8n z&B?{1K1ybGe}VTUJXKxU#-bLp*L66C3Ni@UG&BsX`0Gm5b?>%rjady`vA6at6&N_~ z=z$e6_I;{SX8DbkRdxI~-Vj_{dI1x-J_(~2+cXk_tj|@+7Vi`g4Qx^qbfGh%e5X-r z<(XA+1JM?&OkE*}fN^~kvT*My(C+}UZlMo$W;ianeP1zyKHL1W(*(A1y?>#sg4OzV zm(^ir7C+kb1lmOb)PCP+$25{?80?N1-Y29X$88&+kXdoXwegFg)=N#qRil1SJH|6n z502wR`%J8yLK63l&p&mTR702@SAM&XdQP$zoY76+^cM=23DK8Spbb?1G+LK9E^Jn- z3ief`L`GC=`yb;ADJ2wm{vPfhR7Ax#LpcCVc=VeLp&D&lT-$2Vv|bY1U)tKY9GPWt zSc80-+3?_CCF4gfWDm7<)8!ag?&x!q*m$HFrB|AOE~3hPI^b){`|D-L1HK)jYsd&{ z$CS;r(GfV$T=rdb-l4PMKQjyGNfPz3Lzo-4N*-S>%d3?|Y4FvQ7JAaWati-3I2p0_ zjDl;}TeH0a)?h6ld-QqXSr$fUi;Q4|SS=QW6*Zn>tP=$N5Ac3TG7WbNwmbQ%V%kUB zW0=On4F(ozVZs8_l%hOodaHp+YZUalcIPA^J`Sy5kDp9z5UlRjzUK4s6BZ zg-mP5N3sxS@Kw(>kQFIR3Sim&wyyZq8rDed4}Wc}RmqTc!^JROJQ z`C?1|`rT31lv|Dly9T7TZ&>KlXamVY<-*UddEHON`%f*fy)xiyp83 zVf>+9$Vl(sK)CS_R4(rwqpTK$B-e+I{x&2Z^4<;Sl!>ZEpy^?Lo#cTxI8eaf}-EKHQAPP=s-&^ly~j_u-}> zal5`(!)%asso`MTyc(a?7;3swK~er{%*Is?T{U#gH@7(Juz|k?1N)81mH(ZeL|H#A zC}~uw20AuxkEM@QoE_h8^KEmK&POA90RX?8yWS%okH3h~@z49u{~S9hOS1y~UMs|Y zF&%0>ZBGHK5{RmZn|3=hl71G$VT(^A(3AJX;Og0f9VJ3EONXW)*9Bp?nqZ1`jK_mI{F8EgwSC|0Aa0-SGktE2Nimwgxu zF($*o_96C~c{@0l0uYA^up>0_p7^C3kRbDc!+CL0U}nqfF+wkHG#_~YvG#joS24)wGPPo?>2hn zBw;ox^6Hoe+L`S%=p8wMKr{xk{Z<#GaF+zlCG5dW(cVf0=qk`MtF|vClNdjelA#wz zBeY55dM)n&hJ;h$BP}vI_Mpvm-lV|Gdz!=Uo8rKi{e!dAOH&&4&vohsl%rbRA&{RK zY7oc-0)fBP1}Q|O&T!Z@QdJMXI@Wm~_*1vu*3-BC3$PRY4N2A|iE!!0!KeY%%+CpW zUWLk%=PBddCs*qJMR|*IyritlM0&hxdrS#gzYxm^%E4grak#x6ZBU*6#Kgo(hI6f( zg!(nffY_0%d>@paZEP_cw{js7e}VhX#Y_ z@;*9YDz+w2@5h1%g>^y8YZ@V+?Z_QBM75Qzbd@O1r?x6~A#d)E6J+5RKZJ{CUmVq8 zC4dV%g2V-u!!CUYy77EI>&d(5RC<*-ylKd>ASWDHD9YAal2lZuhzBG^%a?ha_#pRL zZ8~jyWKBftF`wk$!R-f0`UB?ANbMmI#j;RH+bO28E}{VOdoOq5u;tOpqw)hLX4WH6 z=m2usSj@e5zhHl7`e~~b=;;Y1;OX<$2{-vxTlz42urJG^6gv-OJ?kWwFIEf>7;;(3 z<^3z~RREB<{^x*QzhM`$t}MMEE*$jy-htY_kSm5jjeMoO1e?eDjvGgE4GtH_0l@oA z!j9kDR2V;6!?teuj}!PY|BUIYSb+xqLNyj)?&SyOtjB$k_1(8OUFI<{aoNi-C7X96e;4qo~8cuO%jZz?nQxg`}4}lGuW|PF*k;}{AyhU==k}; zAFo%@=Vf*~T)|zuV^VD|Ad2|$k;^O{FZYKB#-@a(K}9T`YPNQ@HW-v;n_DuM_+I== z^|VP!9>%v-qhlywmd|V{t6J~+h^ISfud9gU#+PO0VruE1CoG|>(8bdw1H-LoBtGK= ziJZctV&zx3xbjfvjSRCt%i&kO<#8_Un>2Jp!%4iKP3!YGl_2ZeO$FDU8=%Jfg|Rk( zStmnDHpsZ(+sUJqkC|Vi+M9I1*J~1PIp6=|F`3YfEiU90o^n^mU(eE@a1?-=0XZ z{#!C_Or3(>*UiCcT)s!Ry_?c920vg10xK*9fObrvkZ!SP0hCv>r0$Vnmg_`aGUiCr zV6m(h-c6n#zR?KL@R>kPA7aPl2D^~wjnRfPb7yrMpwP=97FN|^Rjq0iYbbRh8Wr@v z5%M?7zxZ9l)*9j~MuT8GU|Wqzhy%*3GOU7nE2V72??62fEIw>5Zh5&!H%)ICO~dtk z_A>$?{H;8hE8=1;{f4ydk*3=l2}|JGt+^$6QCm>F$LI{bX^CL|e*jM)K`?Mgcbu@? zIpH$BE}Cg}3=LunZ8c@s?CL6ZMN$1=F&UU7GkH_)wKVAn3O|+8Z2y}u`V^eoT0--! z2at-GbtuCg6#krm=~OYW(%HPnPJCQ3Xomimtalh+cf{nYES7$2(A;E+SDe5N5_!?( z65H$z1)R-%xA6PqtR&e%4dEB3w|*QHw;mZf!;Z+va;(bhS@KEQKp+<@-K#sqxw3kL`Bods@T4ER6Ac)Hz8h78vsPe}JY#r9f%|k=5OIC=EOe2)ul_az3&TTH;+&_!;Bot zF==NSuY+qlx+E>QzcH~!o1fwZV!?Q_=UZQQ!F2?$q`a|XpLPdTT`0B!D7SLN%~a^n z5&h${IgboLeznFQwfLMdBbEAax0hRnF5z6{-E zjlO5v3d}=^j)@K%Oyh8p*5aZjFMPSLYd-u;T2{9lQn%FYx^4b!UBHW@_O>WaztMrD zPi?(<n28W%2|O$#?q4F ztE4Blt?}~j^s@Q9*th~dDTj;L+-*fF-?SNb;bDoyK3^yQluvbM9+L}LN$29I9T2zS zItep!%TM>8MGqCMq3e0_fA77t53LGZE~h6CP476O`N7Dol}AV0F18}%FvKGd_No_+ z$o!#45s{T}WCZyLJbD~SoD|Y^JR0R(f0(L$&rsNoqK#SWM#9gO0j=LRPiX{?44|9T zdiieq>Buw(55!(k1|31ueb?|6WN>@aJ^0T%e>k~#%^Htj@lAV#Tr5O~1|P0}A$3|a zVxANt(L(;H=AoK(|Hjn&!(R~gSENx66H(;CIn1(kR*-5P4?Y9n!UP{m>f;?)9n<9E zxIe*0%M4X}AS(%ls#2m;lAw#;RV>yR^MqL4(I`lEP-L!mm^FHeW~Cz7L0h__5`w96AcsY*KDJSW*)s_O_MvIh34t`hcCnCcVfT}Z+341l zvX?_a&mj7^1jHpfhW55`?`xAb?+Ro9c&^tXl}- z({GVHlicKr@EyUx;d#DlFQ_eH)8Gz=n7m#U%14+nc~Xurj3yx|g@@LE@SR7msr`9E xuqYxT#kMoS(wTI5)oxQDgoysn#liTO9+aF8W1v7q5$5?s`r0O1bs8_C{|5jP;!gko literal 0 HcmV?d00001 diff --git a/docs/spelling_wordlist.txt b/docs/spelling_wordlist.txt index 8effe2cc92ca4..6bfca3ef37d59 100644 --- a/docs/spelling_wordlist.txt +++ b/docs/spelling_wordlist.txt @@ -344,6 +344,7 @@ DagRunState DAGs Dask dask +daskexecutor dat Databricks databricks diff --git a/generated/provider_dependencies.json b/generated/provider_dependencies.json index 03d821eff15e9..091a45cf36ee2 100644 --- a/generated/provider_dependencies.json +++ b/generated/provider_dependencies.json @@ -277,6 +277,16 @@ ], "excluded-python-versions": [] }, + "daskexecutor": { + "deps": [ + "apache-airflow>=2.4.0", + "cloudpickle>=1.4.1", + "dask>=2.9.0,!=2022.10.1,!=2023.5.0", + "distributed>=2.11.1,!=2023.5.0" + ], + "cross-providers-deps": [], + "excluded-python-versions": [] + }, "databricks": { "deps": [ "aiohttp>=3.6.3, <4", diff --git a/images/breeze/output-commands-hash.txt b/images/breeze/output-commands-hash.txt index b3fc1ec2265fc..540a5938a00e9 100644 --- a/images/breeze/output-commands-hash.txt +++ b/images/breeze/output-commands-hash.txt @@ -2,7 +2,7 @@ # Please do not solve it but run `breeze setup regenerate-command-images`. # This command should fix the conflict and regenerate help images that you have conflict with. main:344261ca3aa7ff31e098b1d88280566a -build-docs:927336a331afdb9f7b6797c119d9cc16 +build-docs:1baeefeaba160aef5355ffbb09212cf5 ci:fix-ownership:3e5a73533cc96045e72cb258783cfc96 ci:free-space:49af17b032039c05c41a7a8283f365cc ci:get-workflow-info:8246038093359b9c3c110043419473e2 @@ -31,28 +31,28 @@ k8s:status:1b1b6fb6ccc7a211a3eb44568da71659 k8s:tests:2319c6f5ba0681ff7627e3166a1479ef k8s:upload-k8s-image:a9ac79e2f5e3d6b01fa45e764885913f k8s:db08df55a8200bff5858a3b7549bff47 -prod-image:build:58ab53c2f5e799bbe05d6fac26d001bf +prod-image:build:1a321a1243555ab4e2cc95f44e85c92e prod-image:pull:76f1f27e6119928412abecf153fce4bb prod-image:verify:bd2b78738a7c388dbad6076c41a9f906 -prod-image:a434391e2b5201aaeb5c06601485f7bc +prod-image:ad55d9dce93cbfcc1c5ad8640a007340 release-management:add-back-references:8dab6a30076a55f2d31c6d22a94e0ccb release-management:create-minor-branch:a3834afc4aa5d1e98002c9e9e7a9931d release-management:generate-constraints:b8fcaf8f0acd35ed5dbd48659bdb6485 -release-management:generate-issue-content-providers:9cea234261547208ac812464d3e4a598 +release-management:generate-issue-content-providers:6b0d954cb6dbdec0da0a7988feec58f0 release-management:generate-providers-metadata:d4e8e5cfaa024e3963af02d7a873048d release-management:install-provider-packages:a89493be1ae961c13469b5a25a605069 release-management:prepare-airflow-package:85d01c57e5b5ee0fb9e5f9d9706ed3b5 -release-management:prepare-provider-documentation:bc0b95c6a0f308273ab25f728afdb039 -release-management:prepare-provider-packages:3e19a787c71d5515a2bc05539da0022f -release-management:publish-docs:4a7d78a846f7e58c6af4a55cbdf7104b +release-management:prepare-provider-documentation:b8defe8b94bf790d88d055c3e4c0ffba +release-management:prepare-provider-packages:4599a06e636c6fe8fda41b5a751c1429 +release-management:publish-docs:00eae3f9acc2134c14c9a94d789423ce release-management:release-prod-images:4d85a23a2175bf8894de5aedbdd85614 release-management:start-rc-process:b27bd524dd3c89f50a747b60a7e892c1 release-management:start-release:419f48f6a4ff4457cb9de7ff496aebbe release-management:verify-provider-packages:96dce5644aad6b37080acf77b3d8de3a -release-management:e12492cf058a0c32fc6daecde65a1aef -sbom:generate-provider-requirements:1e7609a0ebafb839d30ec90e4254a628 +release-management:f15d70b8cb924cf33e6d7e2a1f32adf8 +sbom:generate-provider-requirements:9abe53200ea5f40e0bf7c27f6087f27f sbom:update-sbom-information:0ce56884e5f842e3e80d6619df1ccc64 -sbom:2ac6b8fc1e84428d5588270eb0d84040 +sbom:935d041028e847d3faf763a95b51063e setup:autocomplete:fffcd49e102e09ccd69b3841a9e3ea8e setup:check-all-params-in-groups:76b3b1d3726ca2a446bab0668b1c50f5 setup:config:38ebaaf93ed42bc7b2a3000eeea2631d diff --git a/images/breeze/output_build-docs.svg b/images/breeze/output_build-docs.svg index f0475ed9c40a9..75e92ebb2ccdf 100644 --- a/images/breeze/output_build-docs.svg +++ b/images/breeze/output_build-docs.svg @@ -1,4 +1,4 @@ - + - + @@ -219,9 +219,12 @@ + + + - Command: build-docs + Command: build-docs @@ -232,64 +235,65 @@ -Usage: breeze build-docs [OPTIONS] +Usage: breeze build-docs [OPTIONS] -╭─ Doc flags ──────────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---docs-only-dOnly build documentation. ---spellcheck-only-sOnly run spell checking. ---clean-buildClean inventories of Inter-Sphinx documentation and generated APIs and sphinx artifacts     -before the build - useful for a clean build.                                                ---one-pass-onlyBuilds documentation in one pass only. This is useful for debugging sphinx errors. ---package-filterList of packages to consider.                                                               -(apache-airflow | apache-airflow-providers-airbyte | apache-airflow-providers-alibaba |     -apache-airflow-providers-amazon | apache-airflow-providers-apache-beam |                    -apache-airflow-providers-apache-cassandra | apache-airflow-providers-apache-drill |         -apache-airflow-providers-apache-druid | apache-airflow-providers-apache-flink |             -apache-airflow-providers-apache-hdfs | apache-airflow-providers-apache-hive |               -apache-airflow-providers-apache-impala | apache-airflow-providers-apache-kafka |            -apache-airflow-providers-apache-kylin | apache-airflow-providers-apache-livy |              -apache-airflow-providers-apache-pig | apache-airflow-providers-apache-pinot |               -apache-airflow-providers-apache-spark | apache-airflow-providers-apache-sqoop |             -apache-airflow-providers-apprise | apache-airflow-providers-arangodb |                      -apache-airflow-providers-asana | apache-airflow-providers-atlassian-jira |                  -apache-airflow-providers-celery | apache-airflow-providers-cloudant |                       -apache-airflow-providers-cncf-kubernetes | apache-airflow-providers-common-sql |            -apache-airflow-providers-databricks | apache-airflow-providers-datadog |                    -apache-airflow-providers-dbt-cloud | apache-airflow-providers-dingding |                    -apache-airflow-providers-discord | apache-airflow-providers-docker |                        -apache-airflow-providers-elasticsearch | apache-airflow-providers-exasol |                  -apache-airflow-providers-facebook | apache-airflow-providers-ftp |                          -apache-airflow-providers-github | apache-airflow-providers-google |                         -apache-airflow-providers-grpc | apache-airflow-providers-hashicorp |                        -apache-airflow-providers-http | apache-airflow-providers-imap |                             -apache-airflow-providers-influxdb | apache-airflow-providers-jdbc |                         -apache-airflow-providers-jenkins | apache-airflow-providers-microsoft-azure |               -apache-airflow-providers-microsoft-mssql | apache-airflow-providers-microsoft-psrp |        -apache-airflow-providers-microsoft-winrm | apache-airflow-providers-mongo |                 -apache-airflow-providers-mysql | apache-airflow-providers-neo4j |                           -apache-airflow-providers-odbc | apache-airflow-providers-openfaas |                         -apache-airflow-providers-openlineage | apache-airflow-providers-opsgenie |                  -apache-airflow-providers-oracle | apache-airflow-providers-pagerduty |                      -apache-airflow-providers-papermill | apache-airflow-providers-plexus |                      -apache-airflow-providers-postgres | apache-airflow-providers-presto |                       -apache-airflow-providers-qubole | apache-airflow-providers-redis |                          -apache-airflow-providers-salesforce | apache-airflow-providers-samba |                      -apache-airflow-providers-segment | apache-airflow-providers-sendgrid |                      -apache-airflow-providers-sftp | apache-airflow-providers-singularity |                      -apache-airflow-providers-slack | apache-airflow-providers-smtp |                            -apache-airflow-providers-snowflake | apache-airflow-providers-sqlite |                      -apache-airflow-providers-ssh | apache-airflow-providers-tableau |                           -apache-airflow-providers-tabular | apache-airflow-providers-telegram |                      -apache-airflow-providers-trino | apache-airflow-providers-vertica |                         -apache-airflow-providers-zendesk | docker-stack | helm-chart)                               ---github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] ---builderBuildx builder used to perform `docker buildx build` commands.(TEXT)[default: autodetect] -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Doc flags ──────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--docs-only-dOnly build documentation. +--spellcheck-only-sOnly run spell checking. +--clean-buildClean inventories of Inter-Sphinx documentation and generated APIs and sphinx artifacts     +before the build - useful for a clean build.                                                +--one-pass-onlyBuilds documentation in one pass only. This is useful for debugging sphinx errors. +--package-filterList of packages to consider.                                                               +(apache-airflow | apache-airflow-providers-airbyte | apache-airflow-providers-alibaba |     +apache-airflow-providers-amazon | apache-airflow-providers-apache-beam |                    +apache-airflow-providers-apache-cassandra | apache-airflow-providers-apache-drill |         +apache-airflow-providers-apache-druid | apache-airflow-providers-apache-flink |             +apache-airflow-providers-apache-hdfs | apache-airflow-providers-apache-hive |               +apache-airflow-providers-apache-impala | apache-airflow-providers-apache-kafka |            +apache-airflow-providers-apache-kylin | apache-airflow-providers-apache-livy |              +apache-airflow-providers-apache-pig | apache-airflow-providers-apache-pinot |               +apache-airflow-providers-apache-spark | apache-airflow-providers-apache-sqoop |             +apache-airflow-providers-apprise | apache-airflow-providers-arangodb |                      +apache-airflow-providers-asana | apache-airflow-providers-atlassian-jira |                  +apache-airflow-providers-celery | apache-airflow-providers-cloudant |                       +apache-airflow-providers-cncf-kubernetes | apache-airflow-providers-common-sql |            +apache-airflow-providers-daskexecutor | apache-airflow-providers-databricks |               +apache-airflow-providers-datadog | apache-airflow-providers-dbt-cloud |                     +apache-airflow-providers-dingding | apache-airflow-providers-discord |                      +apache-airflow-providers-docker | apache-airflow-providers-elasticsearch |                  +apache-airflow-providers-exasol | apache-airflow-providers-facebook |                       +apache-airflow-providers-ftp | apache-airflow-providers-github |                            +apache-airflow-providers-google | apache-airflow-providers-grpc |                           +apache-airflow-providers-hashicorp | apache-airflow-providers-http |                        +apache-airflow-providers-imap | apache-airflow-providers-influxdb |                         +apache-airflow-providers-jdbc | apache-airflow-providers-jenkins |                          +apache-airflow-providers-microsoft-azure | apache-airflow-providers-microsoft-mssql |       +apache-airflow-providers-microsoft-psrp | apache-airflow-providers-microsoft-winrm |        +apache-airflow-providers-mongo | apache-airflow-providers-mysql |                           +apache-airflow-providers-neo4j | apache-airflow-providers-odbc |                            +apache-airflow-providers-openfaas | apache-airflow-providers-openlineage |                  +apache-airflow-providers-opsgenie | apache-airflow-providers-oracle |                       +apache-airflow-providers-pagerduty | apache-airflow-providers-papermill |                   +apache-airflow-providers-plexus | apache-airflow-providers-postgres |                       +apache-airflow-providers-presto | apache-airflow-providers-qubole |                         +apache-airflow-providers-redis | apache-airflow-providers-salesforce |                      +apache-airflow-providers-samba | apache-airflow-providers-segment |                         +apache-airflow-providers-sendgrid | apache-airflow-providers-sftp |                         +apache-airflow-providers-singularity | apache-airflow-providers-slack |                     +apache-airflow-providers-smtp | apache-airflow-providers-snowflake |                        +apache-airflow-providers-sqlite | apache-airflow-providers-ssh |                            +apache-airflow-providers-tableau | apache-airflow-providers-tabular |                       +apache-airflow-providers-telegram | apache-airflow-providers-trino |                        +apache-airflow-providers-vertica | apache-airflow-providers-zendesk | docker-stack |        +helm-chart)                                                                                 +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +--builderBuildx builder used to perform `docker buildx build` commands.(TEXT)[default: autodetect] +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_prod-image.svg b/images/breeze/output_prod-image.svg index 35bf4ae86bb6e..cd179a225efee 100644 --- a/images/breeze/output_prod-image.svg +++ b/images/breeze/output_prod-image.svg @@ -35,8 +35,8 @@ .breeze-prod-image-r1 { fill: #c5c8c6;font-weight: bold } .breeze-prod-image-r2 { fill: #c5c8c6 } .breeze-prod-image-r3 { fill: #d0b344;font-weight: bold } -.breeze-prod-image-r4 { fill: #68a0b3;font-weight: bold } -.breeze-prod-image-r5 { fill: #868887 } +.breeze-prod-image-r4 { fill: #868887 } +.breeze-prod-image-r5 { fill: #68a0b3;font-weight: bold } .breeze-prod-image-r6 { fill: #98a84b;font-weight: bold } @@ -93,18 +93,18 @@ -Usage: breeze prod-image [OPTIONSCOMMAND [ARGS]... +Usage: breeze prod-image [OPTIONS] COMMAND [ARGS]... -Tools that developers can use to manually manage PROD images +Tools that developers can use to manually manage PROD images -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Production Image tools ─────────────────────────────────────────────────────────────────────────────────────────────╮ -build  Build Production image. Include building multiple images for all or selected Python versions sequentially.  -pull   Pull and optionally verify Production images - possibly in parallel for all Python versions.                -verify Verify Production image.                                                                                    -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Production Image tools ─────────────────────────────────────────────────────────────────────────────────────────────╮ +build  Build Production image. Include building multiple images for all or selected Python versions sequentially.  +pull   Pull and optionally verify Production images - possibly in parallel for all Python versions.                +verify Verify Production image.                                                                                    +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_prod-image_build.svg b/images/breeze/output_prod-image_build.svg index 30e4426a06cb4..882393e5d0df9 100644 --- a/images/breeze/output_prod-image_build.svg +++ b/images/breeze/output_prod-image_build.svg @@ -35,8 +35,8 @@ .breeze-prod-image-build-r1 { fill: #c5c8c6;font-weight: bold } .breeze-prod-image-build-r2 { fill: #c5c8c6 } .breeze-prod-image-build-r3 { fill: #d0b344;font-weight: bold } -.breeze-prod-image-build-r4 { fill: #68a0b3;font-weight: bold } -.breeze-prod-image-build-r5 { fill: #868887 } +.breeze-prod-image-build-r4 { fill: #868887 } +.breeze-prod-image-build-r5 { fill: #68a0b3;font-weight: bold } .breeze-prod-image-build-r6 { fill: #98a84b;font-weight: bold } .breeze-prod-image-build-r7 { fill: #8d7b39 } @@ -337,99 +337,99 @@ -Usage: breeze prod-image build [OPTIONS] +Usage: breeze prod-image build [OPTIONS] Build Production image. Include building multiple images for all or selected Python versions sequentially. -╭─ Basic usage ────────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---python-pPython major/minor version used in Airflow image for images. -(>3.8< | 3.9 | 3.10 | 3.11)                                  -[default: 3.8]                                               ---install-airflow-version-VInstall version of Airflow from PyPI.(TEXT) ---upgrade-to-newer-dependencies-uWhen set, upgrade all PIP packages to latest. ---upgrade-on-failureWhen set, attempt to run upgrade to newer dependencies when regular build       -fails.                                                                          ---image-tag-tTag the image after building it.(TEXT)[default: latest] ---tag-as-latestTags the image as latest and update checksum of all files after pulling. Useful -when you build or pull image with --image-tag.                                  ---docker-cache-cCache option for image used during the build.(registry | local | disabled) -[default: registry]                           -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Building images in parallel ────────────────────────────────────────────────────────────────────────────────────────╮ ---run-in-parallelRun the operation in parallel on all or selected subset of Python versions. ---parallelismMaximum number of processes to use while running the operation in parallel. -(INTEGER RANGE)                                                             -[default: 4; 1<=x<=8]                                                       ---python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) -[default: 3.8 3.9 3.10 3.11]                                                   ---skip-cleanupSkip cleanup of temporary files created during parallel run. ---debug-resourcesWhether to show resource information while running in parallel. ---include-success-outputsWhether to include outputs of successful parallel runs (skipped by default). -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Options for customizing images ─────────────────────────────────────────────────────────────────────────────────────╮ ---builderBuildx builder used to perform `docker buildx build` commands.(TEXT) -[default: autodetect]                                          ---install-providers-from-sourcesInstall providers from sources when installing. ---airflow-extrasExtras to install by default.                                                    -(TEXT)                                                                           -[default:                                                                        -aiobotocore,amazon,async,celery,cncf.kubernetes,dask,docker,elasticsearch,ftp,g… ---airflow-constraints-locationIf specified, it is used instead of calculating reference to the constraint      -file. It could be full remote URL to the location file, or local file placed in  -`docker-context-files` (in this case it has to start with                        -/opt/airflow/docker-context-files).                                              -(TEXT)                                                                           ---airflow-constraints-modeMode of constraints for PROD image building.                            -(constraints | constraints-no-providers | constraints-source-providers) -[default: constraints]                                                  ---airflow-constraints-referenceConstraint reference to use when building the image.(TEXT) ---python-imageIf specified this is the base python image used to build the image. Should be    -something like: python:VERSION-slim-bullseye.                                    -(TEXT)                                                                           ---additional-extrasAdditional extra package while installing Airflow in the image.(TEXT) ---additional-pip-install-flagsAdditional flags added to `pip install` commands (except reinstalling `pip`      -itself).                                                                         -(TEXT)                                                                           ---additional-python-depsAdditional python dependencies to use when building the images.(TEXT) ---additional-runtime-apt-depsAdditional apt runtime dependencies to use when building the images.(TEXT) ---additional-runtime-apt-envAdditional environment variables set when adding runtime dependencies.(TEXT) ---additional-runtime-apt-commandAdditional command executed before runtime apt deps are installed.(TEXT) ---additional-dev-apt-depsAdditional apt dev dependencies to use when building the images.(TEXT) ---additional-dev-apt-envAdditional environment variables set when adding dev dependencies.(TEXT) ---additional-dev-apt-commandAdditional command executed before dev apt deps are installed.(TEXT) ---runtime-apt-depsApt runtime dependencies to use when building the images.(TEXT) ---runtime-apt-commandCommand executed before runtime apt deps are installed.(TEXT) ---dev-apt-depsApt dev dependencies to use when building the images.(TEXT) ---dev-apt-commandCommand executed before dev apt deps are installed.(TEXT) -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Customization options (for specific customization needs) ───────────────────────────────────────────────────────────╮ ---install-packages-from-contextInstall wheels from local docker-context-files when building image.        -Implies --disable-airflow-repo-cache.                                      ---cleanup-contextClean up docker context files before running build (cannot be used         -together with --install-packages-from-context).                            ---disable-mysql-client-installationDo not install MySQL client. ---disable-mssql-client-installationDo not install MsSQl client. ---disable-postgres-client-installationDo not install Postgres client. ---disable-airflow-repo-cacheDisable cache from Airflow repository during building. ---install-airflow-referenceInstall Airflow using GitHub tag or branch.(TEXT) ---installation-methodInstall Airflow from: sources or PyPI.(. | apache-airflow) -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Preparing cache and push (for maintainers and CI) ──────────────────────────────────────────────────────────────────╮ ---platformPlatform for Airflow image.(linux/amd64 | linux/arm64 | linux/amd64,linux/arm64) ---pushPush image after building it. ---empty-imagePrepare empty image tagged with the same name as the Airflow image. ---prepare-buildx-cachePrepares build cache (this is done as separate per-platform steps instead of building the  -image).                                                                                    -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Github authentication ──────────────────────────────────────────────────────────────────────────────────────────────╮ ---github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] ---github-tokenThe token used to authenticate to GitHub.(TEXT) -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Basic usage ────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--python-pPython major/minor version used in Airflow image for images. +(>3.8< | 3.9 | 3.10 | 3.11)                                  +[default: 3.8]                                               +--install-airflow-version-VInstall version of Airflow from PyPI.(TEXT) +--upgrade-to-newer-dependencies-uWhen set, upgrade all PIP packages to latest. +--upgrade-on-failureWhen set, attempt to run upgrade to newer dependencies when regular build       +fails.                                                                          +--image-tag-tTag the image after building it.(TEXT)[default: latest] +--tag-as-latestTags the image as latest and update checksum of all files after pulling. Useful +when you build or pull image with --image-tag.                                  +--docker-cache-cCache option for image used during the build.(registry | local | disabled) +[default: registry]                           +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Building images in parallel ────────────────────────────────────────────────────────────────────────────────────────╮ +--run-in-parallelRun the operation in parallel on all or selected subset of Python versions. +--parallelismMaximum number of processes to use while running the operation in parallel. +(INTEGER RANGE)                                                             +[default: 4; 1<=x<=8]                                                       +--python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) +[default: 3.8 3.9 3.10 3.11]                                                   +--skip-cleanupSkip cleanup of temporary files created during parallel run. +--debug-resourcesWhether to show resource information while running in parallel. +--include-success-outputsWhether to include outputs of successful parallel runs (skipped by default). +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Options for customizing images ─────────────────────────────────────────────────────────────────────────────────────╮ +--builderBuildx builder used to perform `docker buildx build` commands.(TEXT) +[default: autodetect]                                          +--install-providers-from-sourcesInstall providers from sources when installing. +--airflow-extrasExtras to install by default.                                                    +(TEXT)                                                                           +[default:                                                                        +aiobotocore,amazon,async,celery,cncf.kubernetes,daskexecutor,docker,elasticsear… +--airflow-constraints-locationIf specified, it is used instead of calculating reference to the constraint      +file. It could be full remote URL to the location file, or local file placed in  +`docker-context-files` (in this case it has to start with                        +/opt/airflow/docker-context-files).                                              +(TEXT)                                                                           +--airflow-constraints-modeMode of constraints for PROD image building.                            +(constraints | constraints-no-providers | constraints-source-providers) +[default: constraints]                                                  +--airflow-constraints-referenceConstraint reference to use when building the image.(TEXT) +--python-imageIf specified this is the base python image used to build the image. Should be    +something like: python:VERSION-slim-bullseye.                                    +(TEXT)                                                                           +--additional-extrasAdditional extra package while installing Airflow in the image.(TEXT) +--additional-pip-install-flagsAdditional flags added to `pip install` commands (except reinstalling `pip`      +itself).                                                                         +(TEXT)                                                                           +--additional-python-depsAdditional python dependencies to use when building the images.(TEXT) +--additional-runtime-apt-depsAdditional apt runtime dependencies to use when building the images.(TEXT) +--additional-runtime-apt-envAdditional environment variables set when adding runtime dependencies.(TEXT) +--additional-runtime-apt-commandAdditional command executed before runtime apt deps are installed.(TEXT) +--additional-dev-apt-depsAdditional apt dev dependencies to use when building the images.(TEXT) +--additional-dev-apt-envAdditional environment variables set when adding dev dependencies.(TEXT) +--additional-dev-apt-commandAdditional command executed before dev apt deps are installed.(TEXT) +--runtime-apt-depsApt runtime dependencies to use when building the images.(TEXT) +--runtime-apt-commandCommand executed before runtime apt deps are installed.(TEXT) +--dev-apt-depsApt dev dependencies to use when building the images.(TEXT) +--dev-apt-commandCommand executed before dev apt deps are installed.(TEXT) +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Customization options (for specific customization needs) ───────────────────────────────────────────────────────────╮ +--install-packages-from-contextInstall wheels from local docker-context-files when building image.        +Implies --disable-airflow-repo-cache.                                      +--cleanup-contextClean up docker context files before running build (cannot be used         +together with --install-packages-from-context).                            +--disable-mysql-client-installationDo not install MySQL client. +--disable-mssql-client-installationDo not install MsSQl client. +--disable-postgres-client-installationDo not install Postgres client. +--disable-airflow-repo-cacheDisable cache from Airflow repository during building. +--install-airflow-referenceInstall Airflow using GitHub tag or branch.(TEXT) +--installation-methodInstall Airflow from: sources or PyPI.(. | apache-airflow) +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Preparing cache and push (for maintainers and CI) ──────────────────────────────────────────────────────────────────╮ +--platformPlatform for Airflow image.(linux/amd64 | linux/arm64 | linux/amd64,linux/arm64) +--pushPush image after building it. +--empty-imagePrepare empty image tagged with the same name as the Airflow image. +--prepare-buildx-cachePrepares build cache (this is done as separate per-platform steps instead of building the  +image).                                                                                    +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Github authentication ──────────────────────────────────────────────────────────────────────────────────────────────╮ +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +--github-tokenThe token used to authenticate to GitHub.(TEXT) +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_release-management.svg b/images/breeze/output_release-management.svg new file mode 100644 index 0000000000000..c25e978e3c16a --- /dev/null +++ b/images/breeze/output_release-management.svg @@ -0,0 +1,171 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Command: release-management + + + + + + + + + + +Usage: breeze release-management [OPTIONS] COMMAND [ARGS]... + +Tools that release managers can use to prepare and manage Airflow releases + +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Airflow release commands ───────────────────────────────────────────────────────────────────────────────────────────╮ +prepare-airflow-package      Prepare sdist/whl package of Airflow.                                                 +create-minor-branch          Create a new version branch and update the default branches in main                   +start-rc-process             Start RC process                                                                      +start-release                Start Airflow release process                                                         +release-prod-images          Release production images to DockerHub (needs DockerHub permissions).                 +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Providers release commands ─────────────────────────────────────────────────────────────────────────────────────────╮ +prepare-provider-documentation      Prepare CHANGELOG, README and COMMITS information for providers.               +prepare-provider-packages           Prepare sdist/whl packages of Airflow Providers.                               +install-provider-packages           Installs provider packages that can be found in dist.                          +verify-provider-packages            Verifies if all provider code is following expectations for providers.         +generate-providers-metadata         Generates metadata for providers.                                              +generate-issue-content-providers    Generates content for issue to test the release.                               +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Other release commands ─────────────────────────────────────────────────────────────────────────────────────────────╮ +publish-docs            Command to publish generated documentation to airflow-site                                 +generate-constraints    Generates pinned constraint files with all extras from setup.py in parallel.               +add-back-references     Command to add back references for documentation to make it backward compatible            +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + + + + diff --git a/images/breeze/output_release-management_generate-issue-content-providers.svg b/images/breeze/output_release-management_generate-issue-content-providers.svg index 12eabb09189c4..04fd3c8155f25 100644 --- a/images/breeze/output_release-management_generate-issue-content-providers.svg +++ b/images/breeze/output_release-management_generate-issue-content-providers.svg @@ -1,4 +1,4 @@ - + - + @@ -150,9 +150,12 @@ + + + - Command: release-management generate-issue-content-providers + Command: release-management generate-issue-content-providers @@ -163,41 +166,42 @@ -Usage: breeze release-management generate-issue-content-providers [OPTIONS] [airbyte | alibaba | amazon | apache.beam +Usage: breeze release-management generate-issue-content-providers [OPTIONS] [airbyte | alibaba | amazon | apache.beam                                                                   | apache.cassandra | apache.drill | apache.druid |                                                                   apache.flink | apache.hdfs | apache.hive |                                                                   apache.impala | apache.kafka | apache.kylin |                                                                   apache.livy | apache.pig | apache.pinot |                                                                   apache.spark | apache.sqoop | apprise | arangodb |                                                                   asana | atlassian.jira | celery | cloudant | -                                                                  cncf.kubernetes | common.sql | databricks | datadog -                                                                  | dbt.cloud | dingding | discord | docker | -                                                                  elasticsearch | exasol | facebook | ftp | github | -                                                                  google | grpc | hashicorp | http | imap | influxdb | -                                                                  jdbc | jenkins | microsoft.azure | microsoft.mssql | -                                                                  microsoft.psrp | microsoft.winrm | mongo | mysql | -                                                                  neo4j | odbc | openfaas | openlineage | opsgenie | -                                                                  oracle | pagerduty | papermill | plexus | postgres | -                                                                  presto | qubole | redis | salesforce | samba | -                                                                  segment | sendgrid | sftp | singularity | slack | -                                                                  smtp | snowflake | sqlite | ssh | tableau | tabular -                                                                  | telegram | trino | vertica | zendesk]... - -Generates content for issue to test the release. - -╭─ Generate issue content flags ───────────────────────────────────────────────────────────────────────────────────────╮ ---github-tokenGitHub token used to authenticate. You can set omit it if you have GITHUB_TOKEN env      -variable set. Can be generated with:                                                     -https://github.com/settings/tokens/new?description=Read%20sssues&scopes=repo:status      -(TEXT)                                                                                   ---suffixSuffix to add to the version prepared(TEXT) ---only-available-in-distOnly consider package ids with packages prepared in the dist folder ---excluded-pr-listComa-separated list of PRs to exclude from the issue.(TEXT) ---disable-progressDisable progress bar -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +                                                                  cncf.kubernetes | common.sql | daskexecutor | +                                                                  databricks | datadog | dbt.cloud | dingding | +                                                                  discord | docker | elasticsearch | exasol | facebook +                                                                  | ftp | github | google | grpc | hashicorp | http | +                                                                  imap | influxdb | jdbc | jenkins | microsoft.azure | +                                                                  microsoft.mssql | microsoft.psrp | microsoft.winrm | +                                                                  mongo | mysql | neo4j | odbc | openfaas | +                                                                  openlineage | opsgenie | oracle | pagerduty | +                                                                  papermill | plexus | postgres | presto | qubole | +                                                                  redis | salesforce | samba | segment | sendgrid | +                                                                  sftp | singularity | slack | smtp | snowflake | +                                                                  sqlite | ssh | tableau | tabular | telegram | trino +                                                                  | vertica | zendesk]... + +Generates content for issue to test the release. + +╭─ Generate issue content flags ───────────────────────────────────────────────────────────────────────────────────────╮ +--github-tokenGitHub token used to authenticate. You can set omit it if you have GITHUB_TOKEN env      +variable set. Can be generated with:                                                     +https://github.com/settings/tokens/new?description=Read%20sssues&scopes=repo:status      +(TEXT)                                                                                   +--suffixSuffix to add to the version prepared(TEXT) +--only-available-in-distOnly consider package ids with packages prepared in the dist folder +--excluded-pr-listComa-separated list of PRs to exclude from the issue.(TEXT) +--disable-progressDisable progress bar +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_release-management_prepare-provider-documentation.svg b/images/breeze/output_release-management_prepare-provider-documentation.svg index 69b5e0589ba80..68c55935ffd9b 100644 --- a/images/breeze/output_release-management_prepare-provider-documentation.svg +++ b/images/breeze/output_release-management_prepare-provider-documentation.svg @@ -35,8 +35,8 @@ .breeze-release-management-prepare-provider-documentation-r1 { fill: #c5c8c6;font-weight: bold } .breeze-release-management-prepare-provider-documentation-r2 { fill: #c5c8c6 } .breeze-release-management-prepare-provider-documentation-r3 { fill: #d0b344;font-weight: bold } -.breeze-release-management-prepare-provider-documentation-r4 { fill: #68a0b3;font-weight: bold } -.breeze-release-management-prepare-provider-documentation-r5 { fill: #868887 } +.breeze-release-management-prepare-provider-documentation-r4 { fill: #868887 } +.breeze-release-management-prepare-provider-documentation-r5 { fill: #68a0b3;font-weight: bold } .breeze-release-management-prepare-provider-documentation-r6 { fill: #98a84b;font-weight: bold } .breeze-release-management-prepare-provider-documentation-r7 { fill: #8d7b39 } @@ -175,45 +175,45 @@ -Usage: breeze release-management prepare-provider-documentation [OPTIONS] [airbyte | alibaba | amazon | apache.beam | +Usage: breeze release-management prepare-provider-documentation [OPTIONS] [airbyte | alibaba | amazon | apache.beam |                                                                 apache.cassandra | apache.drill | apache.druid |                                                                 apache.flink | apache.hdfs | apache.hive |                                                                 apache.impala | apache.kafka | apache.kylin |                                                                 apache.livy | apache.pig | apache.pinot | apache.spark                                                                 | apache.sqoop | apprise | arangodb | asana |                                                                 atlassian.jira | celery | cloudant | cncf.kubernetes | -                                                                common.sql | databricks | datadog | dbt.cloud | -                                                                dingding | discord | docker | elasticsearch | exasol | -                                                                facebook | ftp | github | google | grpc | hashicorp | -                                                                http | imap | influxdb | jdbc | jenkins | -                                                                microsoft.azure | microsoft.mssql | microsoft.psrp | -                                                                microsoft.winrm | mongo | mysql | neo4j | odbc | -                                                                openfaas | openlineage | opsgenie | oracle | pagerduty -                                                                | papermill | plexus | postgres | presto | qubole | -                                                                redis | salesforce | samba | segment | sendgrid | sftp -                                                                | singularity | slack | smtp | snowflake | sqlite | -                                                                ssh | tableau | tabular | telegram | trino | vertica | -                                                                zendesk]... +                                                                common.sql | daskexecutor | databricks | datadog | +                                                                dbt.cloud | dingding | discord | docker | +                                                                elasticsearch | exasol | facebook | ftp | github | +                                                                google | grpc | hashicorp | http | imap | influxdb | +                                                                jdbc | jenkins | microsoft.azure | microsoft.mssql | +                                                                microsoft.psrp | microsoft.winrm | mongo | mysql | +                                                                neo4j | odbc | openfaas | openlineage | opsgenie | +                                                                oracle | pagerduty | papermill | plexus | postgres | +                                                                presto | qubole | redis | salesforce | samba | segment +                                                                | sendgrid | sftp | singularity | slack | smtp | +                                                                snowflake | sqlite | ssh | tableau | tabular | +                                                                telegram | trino | vertica | zendesk]... -Prepare CHANGELOGREADME and COMMITS information for providers. +Prepare CHANGELOG, README and COMMITS information for providers. -╭─ Provider documentation preparation flags ───────────────────────────────────────────────────────────────────────────╮ ---debugDrop user in shell instead of running the command. Useful for debugging. ---github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] ---base-branchBase branch to use as diff for documentation generation (used for releasing from old  -branch)                                                                               -(TEXT)                                                                                ---only-min-version-updateOnly update minimum version in __init__.py files and regenerate corresponding         -documentation                                                                         ---regenerate-missing-docsOnly regenerate missing documentation, do not bump version. Useful if templates were  -added and you need to regenerate documentation.                                       -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---answer-aForce answer to questions.(y | n | q | yes | no | quit) ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Provider documentation preparation flags ───────────────────────────────────────────────────────────────────────────╮ +--debugDrop user in shell instead of running the command. Useful for debugging. +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +--base-branchBase branch to use as diff for documentation generation (used for releasing from old  +branch)                                                                               +(TEXT)                                                                                +--only-min-version-updateOnly update minimum version in __init__.py files and regenerate corresponding         +documentation                                                                         +--regenerate-missing-docsOnly regenerate missing documentation, do not bump version. Useful if templates were  +added and you need to regenerate documentation.                                       +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--answer-aForce answer to questions.(y | n | q | yes | no | quit) +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_release-management_prepare-provider-packages.svg b/images/breeze/output_release-management_prepare-provider-packages.svg index e205331cb588e..5a42796427fe7 100644 --- a/images/breeze/output_release-management_prepare-provider-packages.svg +++ b/images/breeze/output_release-management_prepare-provider-packages.svg @@ -35,8 +35,8 @@ .breeze-release-management-prepare-provider-packages-r1 { fill: #c5c8c6;font-weight: bold } .breeze-release-management-prepare-provider-packages-r2 { fill: #c5c8c6 } .breeze-release-management-prepare-provider-packages-r3 { fill: #d0b344;font-weight: bold } -.breeze-release-management-prepare-provider-packages-r4 { fill: #68a0b3;font-weight: bold } -.breeze-release-management-prepare-provider-packages-r5 { fill: #868887 } +.breeze-release-management-prepare-provider-packages-r4 { fill: #868887 } +.breeze-release-management-prepare-provider-packages-r5 { fill: #68a0b3;font-weight: bold } .breeze-release-management-prepare-provider-packages-r6 { fill: #8d7b39 } .breeze-release-management-prepare-provider-packages-r7 { fill: #98a84b;font-weight: bold } @@ -154,17 +154,17 @@ -Usage: breeze release-management prepare-provider-packages [OPTIONS] [airbyte | alibaba | amazon | apache.beam | +Usage: breeze release-management prepare-provider-packages [OPTIONS] [airbyte | alibaba | amazon | apache.beam |                                                            apache.cassandra | apache.drill | apache.druid |                                                            apache.flink | apache.hdfs | apache.hive | apache.impala |                                                            apache.kafka | apache.kylin | apache.livy | apache.pig |                                                            apache.pinot | apache.spark | apache.sqoop | apprise |                                                            arangodb | asana | atlassian.jira | celery | cloudant | -                                                           cncf.kubernetes | common.sql | databricks | datadog | -                                                           dbt.cloud | dingding | discord | docker | elasticsearch | -                                                           exasol | facebook | ftp | github | google | grpc | -                                                           hashicorp | http | imap | influxdb | jdbc | jenkins | -                                                           microsoft.azure | microsoft.mssql | microsoft.psrp | +                                                           cncf.kubernetes | common.sql | daskexecutor | databricks | +                                                           datadog | dbt.cloud | dingding | discord | docker | +                                                           elasticsearch | exasol | facebook | ftp | github | google | +                                                           grpc | hashicorp | http | imap | influxdb | jdbc | jenkins +                                                           | microsoft.azure | microsoft.mssql | microsoft.psrp |                                                            microsoft.winrm | mongo | mysql | neo4j | odbc | openfaas |                                                            openlineage | opsgenie | oracle | pagerduty | papermill |                                                            plexus | postgres | presto | qubole | redis | salesforce | @@ -174,18 +174,18 @@ Prepare sdist/whl packages of Airflow Providers. -╭─ Package flags ──────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---package-formatFormat of packages.(wheel | sdist | both)[default: wheel] ---version-suffix-for-pypiVersion suffix used for PyPI packages (alpha, beta, rc1, etc.).(TEXT) ---package-list-fileRead list of packages from text file (one package per line).(FILENAME) ---debugDrop user in shell instead of running the command. Useful for debugging. ---github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Package flags ──────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--package-formatFormat of packages.(wheel | sdist | both)[default: wheel] +--version-suffix-for-pypiVersion suffix used for PyPI packages (alpha, beta, rc1, etc.).(TEXT) +--package-list-fileRead list of packages from text file (one package per line).(FILENAME) +--debugDrop user in shell instead of running the command. Useful for debugging. +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_release-management_publish-docs.svg b/images/breeze/output_release-management_publish-docs.svg index f534f04e953ce..a118eae74f03d 100644 --- a/images/breeze/output_release-management_publish-docs.svg +++ b/images/breeze/output_release-management_publish-docs.svg @@ -35,8 +35,8 @@ .breeze-release-management-publish-docs-r1 { fill: #c5c8c6;font-weight: bold } .breeze-release-management-publish-docs-r2 { fill: #c5c8c6 } .breeze-release-management-publish-docs-r3 { fill: #d0b344;font-weight: bold } -.breeze-release-management-publish-docs-r4 { fill: #68a0b3;font-weight: bold } -.breeze-release-management-publish-docs-r5 { fill: #868887 } +.breeze-release-management-publish-docs-r4 { fill: #868887 } +.breeze-release-management-publish-docs-r5 { fill: #68a0b3;font-weight: bold } .breeze-release-management-publish-docs-r6 { fill: #98a84b;font-weight: bold } .breeze-release-management-publish-docs-r7 { fill: #8d7b39 } .breeze-release-management-publish-docs-r8 { fill: #cc555a } @@ -228,62 +228,62 @@ -Usage: breeze release-management publish-docs [OPTIONS] +Usage: breeze release-management publish-docs [OPTIONS] Command to publish generated documentation to airflow-site -╭─ Publish Docs ───────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---override-versioned-sOverrides versioned directories. ---package-filterList of packages to consider.                                                       -(apache-airflow | apache-airflow-providers-airbyte |                                -apache-airflow-providers-alibaba | apache-airflow-providers-amazon |                -apache-airflow-providers-apache-beam | apache-airflow-providers-apache-cassandra |  -apache-airflow-providers-apache-drill | apache-airflow-providers-apache-druid |     -apache-airflow-providers-apache-flink | apache-airflow-providers-apache-hdfs |      -apache-airflow-providers-apache-hive | apache-airflow-providers-apache-impala |     -apache-airflow-providers-apache-kafka | apache-airflow-providers-apache-kylin |     -apache-airflow-providers-apache-livy | apache-airflow-providers-apache-pig |        -apache-airflow-providers-apache-pinot | apache-airflow-providers-apache-spark |     -apache-airflow-providers-apache-sqoop | apache-airflow-providers-apprise |          -apache-airflow-providers-arangodb | apache-airflow-providers-asana |                -apache-airflow-providers-atlassian-jira | apache-airflow-providers-celery |         -apache-airflow-providers-cloudant | apache-airflow-providers-cncf-kubernetes |      -apache-airflow-providers-common-sql | apache-airflow-providers-databricks |         -apache-airflow-providers-datadog | apache-airflow-providers-dbt-cloud |             -apache-airflow-providers-dingding | apache-airflow-providers-discord |              -apache-airflow-providers-docker | apache-airflow-providers-elasticsearch |          -apache-airflow-providers-exasol | apache-airflow-providers-facebook |               -apache-airflow-providers-ftp | apache-airflow-providers-github |                    -apache-airflow-providers-google | apache-airflow-providers-grpc |                   -apache-airflow-providers-hashicorp | apache-airflow-providers-http |                -apache-airflow-providers-imap | apache-airflow-providers-influxdb |                 -apache-airflow-providers-jdbc | apache-airflow-providers-jenkins |                  -apache-airflow-providers-microsoft-azure | apache-airflow-providers-microsoft-mssql -| apache-airflow-providers-microsoft-psrp |                                         -apache-airflow-providers-microsoft-winrm | apache-airflow-providers-mongo |         -apache-airflow-providers-mysql | apache-airflow-providers-neo4j |                   -apache-airflow-providers-odbc | apache-airflow-providers-openfaas |                 -apache-airflow-providers-openlineage | apache-airflow-providers-opsgenie |          -apache-airflow-providers-oracle | apache-airflow-providers-pagerduty |              -apache-airflow-providers-papermill | apache-airflow-providers-plexus |              -apache-airflow-providers-postgres | apache-airflow-providers-presto |               -apache-airflow-providers-qubole | apache-airflow-providers-redis |                  -apache-airflow-providers-salesforce | apache-airflow-providers-samba |              -apache-airflow-providers-segment | apache-airflow-providers-sendgrid |              -apache-airflow-providers-sftp | apache-airflow-providers-singularity |              -apache-airflow-providers-slack | apache-airflow-providers-smtp |                    -apache-airflow-providers-snowflake | apache-airflow-providers-sqlite |              -apache-airflow-providers-ssh | apache-airflow-providers-tableau |                   -apache-airflow-providers-tabular | apache-airflow-providers-telegram |              -apache-airflow-providers-trino | apache-airflow-providers-vertica |                 -apache-airflow-providers-zendesk | docker-stack | helm-chart)                       -*--airflow-site-directory-aLocal directory path of cloned airflow-site repo.(TEXT)[required] -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Publish Docs ───────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--override-versioned-sOverrides versioned directories. +--package-filterList of packages to consider.                                                       +(apache-airflow | apache-airflow-providers-airbyte |                                +apache-airflow-providers-alibaba | apache-airflow-providers-amazon |                +apache-airflow-providers-apache-beam | apache-airflow-providers-apache-cassandra |  +apache-airflow-providers-apache-drill | apache-airflow-providers-apache-druid |     +apache-airflow-providers-apache-flink | apache-airflow-providers-apache-hdfs |      +apache-airflow-providers-apache-hive | apache-airflow-providers-apache-impala |     +apache-airflow-providers-apache-kafka | apache-airflow-providers-apache-kylin |     +apache-airflow-providers-apache-livy | apache-airflow-providers-apache-pig |        +apache-airflow-providers-apache-pinot | apache-airflow-providers-apache-spark |     +apache-airflow-providers-apache-sqoop | apache-airflow-providers-apprise |          +apache-airflow-providers-arangodb | apache-airflow-providers-asana |                +apache-airflow-providers-atlassian-jira | apache-airflow-providers-celery |         +apache-airflow-providers-cloudant | apache-airflow-providers-cncf-kubernetes |      +apache-airflow-providers-common-sql | apache-airflow-providers-daskexecutor |       +apache-airflow-providers-databricks | apache-airflow-providers-datadog |            +apache-airflow-providers-dbt-cloud | apache-airflow-providers-dingding |            +apache-airflow-providers-discord | apache-airflow-providers-docker |                +apache-airflow-providers-elasticsearch | apache-airflow-providers-exasol |          +apache-airflow-providers-facebook | apache-airflow-providers-ftp |                  +apache-airflow-providers-github | apache-airflow-providers-google |                 +apache-airflow-providers-grpc | apache-airflow-providers-hashicorp |                +apache-airflow-providers-http | apache-airflow-providers-imap |                     +apache-airflow-providers-influxdb | apache-airflow-providers-jdbc |                 +apache-airflow-providers-jenkins | apache-airflow-providers-microsoft-azure |       +apache-airflow-providers-microsoft-mssql | apache-airflow-providers-microsoft-psrp  +| apache-airflow-providers-microsoft-winrm | apache-airflow-providers-mongo |       +apache-airflow-providers-mysql | apache-airflow-providers-neo4j |                   +apache-airflow-providers-odbc | apache-airflow-providers-openfaas |                 +apache-airflow-providers-openlineage | apache-airflow-providers-opsgenie |          +apache-airflow-providers-oracle | apache-airflow-providers-pagerduty |              +apache-airflow-providers-papermill | apache-airflow-providers-plexus |              +apache-airflow-providers-postgres | apache-airflow-providers-presto |               +apache-airflow-providers-qubole | apache-airflow-providers-redis |                  +apache-airflow-providers-salesforce | apache-airflow-providers-samba |              +apache-airflow-providers-segment | apache-airflow-providers-sendgrid |              +apache-airflow-providers-sftp | apache-airflow-providers-singularity |              +apache-airflow-providers-slack | apache-airflow-providers-smtp |                    +apache-airflow-providers-snowflake | apache-airflow-providers-sqlite |              +apache-airflow-providers-ssh | apache-airflow-providers-tableau |                   +apache-airflow-providers-tabular | apache-airflow-providers-telegram |              +apache-airflow-providers-trino | apache-airflow-providers-vertica |                 +apache-airflow-providers-zendesk | docker-stack | helm-chart)                       +*--airflow-site-directory-aLocal directory path of cloned airflow-site repo.(TEXT)[required] +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_sbom.svg b/images/breeze/output_sbom.svg index 205f0fa242690..0f42e678c1799 100644 --- a/images/breeze/output_sbom.svg +++ b/images/breeze/output_sbom.svg @@ -35,8 +35,8 @@ .breeze-sbom-r1 { fill: #c5c8c6;font-weight: bold } .breeze-sbom-r2 { fill: #c5c8c6 } .breeze-sbom-r3 { fill: #d0b344;font-weight: bold } -.breeze-sbom-r4 { fill: #68a0b3;font-weight: bold } -.breeze-sbom-r5 { fill: #868887 } +.breeze-sbom-r4 { fill: #868887 } +.breeze-sbom-r5 { fill: #68a0b3;font-weight: bold } .breeze-sbom-r6 { fill: #98a84b;font-weight: bold } @@ -90,17 +90,17 @@ -Usage: breeze sbom [OPTIONSCOMMAND [ARGS]... +Usage: breeze sbom [OPTIONS] COMMAND [ARGS]... Tools that release managers can use to prepare sbom information -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ SBOM commands ──────────────────────────────────────────────────────────────────────────────────────────────────────╮ -update-sbom-information                     Update SBOM information in airflow-site project.                       -generate-provider-requirements              Generate requirements for selected provider.                           -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ SBOM commands ──────────────────────────────────────────────────────────────────────────────────────────────────────╮ +update-sbom-information                     Update SBOM information in airflow-site project.                       +generate-provider-requirements              Generate requirements for selected provider.                           +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_sbom_generate-provider-requirements.svg b/images/breeze/output_sbom_generate-provider-requirements.svg index a803d93f6d92e..9dc8ae664aa2a 100644 --- a/images/breeze/output_sbom_generate-provider-requirements.svg +++ b/images/breeze/output_sbom_generate-provider-requirements.svg @@ -35,8 +35,8 @@ .breeze-sbom-generate-provider-requirements-r1 { fill: #c5c8c6;font-weight: bold } .breeze-sbom-generate-provider-requirements-r2 { fill: #c5c8c6 } .breeze-sbom-generate-provider-requirements-r3 { fill: #d0b344;font-weight: bold } -.breeze-sbom-generate-provider-requirements-r4 { fill: #68a0b3;font-weight: bold } -.breeze-sbom-generate-provider-requirements-r5 { fill: #868887 } +.breeze-sbom-generate-provider-requirements-r4 { fill: #868887 } +.breeze-sbom-generate-provider-requirements-r5 { fill: #68a0b3;font-weight: bold } .breeze-sbom-generate-provider-requirements-r6 { fill: #8d7b39 } .breeze-sbom-generate-provider-requirements-r7 { fill: #cc555a } .breeze-sbom-generate-provider-requirements-r8 { fill: #8a4346 } @@ -144,34 +144,34 @@ -Usage: breeze sbom generate-provider-requirements [OPTIONS] +Usage: breeze sbom generate-provider-requirements [OPTIONS] Generate requirements for selected provider. -╭─ Generate provider requirements flags ───────────────────────────────────────────────────────────────────────────────╮ ---airflow-versionAirflow version to use to generate the requirements(TEXT) ---pythonPython version to generate the requirements for(3.6 | 3.7 | 3.8 | 3.9 | 3.10 | 3.11) -*--provider-idProvider to generate the requirements for                                                   -(airbyte | alibaba | amazon | apache.beam | apache.cassandra | apache.drill | apache.druid  -| apache.flink | apache.hdfs | apache.hive | apache.impala | apache.kafka | apache.kylin |  -apache.livy | apache.pig | apache.pinot | apache.spark | apache.sqoop | apprise | arangodb  -| asana | atlassian.jira | celery | cloudant | cncf.kubernetes | common.sql | databricks |  -datadog | dbt.cloud | dingding | discord | docker | elasticsearch | exasol | facebook | ftp -| github | google | grpc | hashicorp | http | imap | influxdb | jdbc | jenkins |            -microsoft.azure | microsoft.mssql | microsoft.psrp | microsoft.winrm | mongo | mysql |      -neo4j | odbc | openfaas | openlineage | opsgenie | oracle | pagerduty | papermill | plexus  -| postgres | presto | qubole | redis | salesforce | samba | segment | sendgrid | sftp |     -singularity | slack | smtp | snowflake | sqlite | ssh | tableau | tabular | telegram |      -trino | vertica | zendesk)                                                                  -[required]                                                                                  ---provider-versionProvider version to generate the requirements for(TEXT) -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---answer-aForce answer to questions.(y | n | q | yes | no | quit) ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Generate provider requirements flags ───────────────────────────────────────────────────────────────────────────────╮ +--airflow-versionAirflow version to use to generate the requirements(TEXT) +--pythonPython version to generate the requirements for(3.6 | 3.7 | 3.8 | 3.9 | 3.10 | 3.11) +*--provider-idProvider to generate the requirements for                                                   +(airbyte | alibaba | amazon | apache.beam | apache.cassandra | apache.drill | apache.druid  +| apache.flink | apache.hdfs | apache.hive | apache.impala | apache.kafka | apache.kylin |  +apache.livy | apache.pig | apache.pinot | apache.spark | apache.sqoop | apprise | arangodb  +| asana | atlassian.jira | celery | cloudant | cncf.kubernetes | common.sql | daskexecutor  +| databricks | datadog | dbt.cloud | dingding | discord | docker | elasticsearch | exasol | +facebook | ftp | github | google | grpc | hashicorp | http | imap | influxdb | jdbc |       +jenkins | microsoft.azure | microsoft.mssql | microsoft.psrp | microsoft.winrm | mongo |    +mysql | neo4j | odbc | openfaas | openlineage | opsgenie | oracle | pagerduty | papermill | +plexus | postgres | presto | qubole | redis | salesforce | samba | segment | sendgrid |     +sftp | singularity | slack | smtp | snowflake | sqlite | ssh | tableau | tabular | telegram +| trino | vertica | zendesk)                                                                +[required]                                                                                  +--provider-versionProvider version to generate the requirements for(TEXT) +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--answer-aForce answer to questions.(y | n | q | yes | no | quit) +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/scripts/ci/installed_providers.txt b/scripts/ci/installed_providers.txt index 3d511abfcc961..9cdcf765eb06a 100644 --- a/scripts/ci/installed_providers.txt +++ b/scripts/ci/installed_providers.txt @@ -2,6 +2,7 @@ amazon celery cncf.kubernetes common.sql +daskexecutor docker elasticsearch ftp diff --git a/scripts/ci/pre_commit/pre_commit_insert_extras.py b/scripts/ci/pre_commit/pre_commit_insert_extras.py index 2750e9dbaed80..2ec53c1962c7f 100755 --- a/scripts/ci/pre_commit/pre_commit_insert_extras.py +++ b/scripts/ci/pre_commit/pre_commit_insert_extras.py @@ -45,7 +45,7 @@ CONSTANTS_FOOTER = "# END EXTRAS HERE" DEFAULT_EXTRAS = ( - "amazon,async,celery,cncf.kubernetes,dask,docker,elasticsearch,ftp,google," + "amazon,async,celery,cncf.kubernetes,daskexecutor,docker,elasticsearch,ftp,google," "google_auth,grpc,hashicorp,http,ldap,microsoft.azure,mysql,odbc,pandas," "postgres,redis,sendgrid,sftp,slack,ssh,statsd,virtualenv" ) diff --git a/setup.py b/setup.py index 44998d8747ce9..837ef85513eb9 100644 --- a/setup.py +++ b/setup.py @@ -481,10 +481,10 @@ def get_unique_dependency_list(req_list_iterable: Iterable[list[str]]): CORE_EXTRAS_DEPENDENCIES: dict[str, list[str]] = { "aiobotocore": aiobotocore, "async": async_packages, - "celery": celery, + "celery": celery, # TODO: remove and move to a regular provider package in a separate PR "cgroups": cgroups, - "cncf.kubernetes": kubernetes, - "dask": dask, + "cncf.kubernetes": kubernetes, # TODO: remove and move to a regular provider package in a separate PR + "dask": dask, # TODO: remove and move to a provider package in a separate PR "deprecated_api": deprecated_api, "github_enterprise": flask_appbuilder_oauth, "google_auth": flask_appbuilder_oauth, @@ -538,6 +538,7 @@ def add_additional_extras() -> None: "azure": "microsoft.azure", "cassandra": "apache.cassandra", "crypto": "", # this is legacy extra - all dependencies are already "install-requires" + "dask": "daskexecutor", "druid": "apache.druid", "gcp": "google", "gcp_api": "google", @@ -844,7 +845,7 @@ def replace_extra_dependencies_with_provider_packages(extra: str, providers: lis :param extra: Name of the extra to add providers to :param providers: list of provider ids """ - if extra in ["cncf.kubernetes", "kubernetes", "celery"]: + if extra in ["cncf.kubernetes", "kubernetes", "celery", "daskexecutor", "dask"]: EXTRAS_DEPENDENCIES[extra].extend( [get_provider_package_name_from_package_id(package_name) for package_name in providers] ) diff --git a/tests/providers/daskexecutor/__init__.py b/tests/providers/daskexecutor/__init__.py new file mode 100644 index 0000000000000..13a83393a9124 --- /dev/null +++ b/tests/providers/daskexecutor/__init__.py @@ -0,0 +1,16 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. diff --git a/tests/executors/test_dask_executor.py b/tests/providers/daskexecutor/test_dask_executor.py similarity index 98% rename from tests/executors/test_dask_executor.py rename to tests/providers/daskexecutor/test_dask_executor.py index d8ded41e158bb..2d559eaa404ef 100644 --- a/tests/executors/test_dask_executor.py +++ b/tests/providers/daskexecutor/test_dask_executor.py @@ -24,10 +24,10 @@ from distributed import LocalCluster from airflow.exceptions import AirflowException -from airflow.executors.dask_executor import DaskExecutor from airflow.jobs.backfill_job_runner import BackfillJobRunner from airflow.jobs.job import Job, run_job from airflow.models import DagBag +from airflow.providers.daskexecutor.executors.dask_executor import DaskExecutor from airflow.utils import timezone from tests.test_utils.config import conf_vars @@ -159,7 +159,7 @@ def test_tls(self): # and tasks to have completed. executor.client.close() - @mock.patch("airflow.executors.dask_executor.DaskExecutor.sync") + @mock.patch("airflow.providers.daskexecutor.executors.dask_executor.DaskExecutor.sync") @mock.patch("airflow.executors.base_executor.BaseExecutor.trigger_tasks") @mock.patch("airflow.executors.base_executor.Stats.gauge") def test_gauge_executor_metrics(self, mock_stats_gauge, mock_trigger_tasks, mock_sync):