Skip to content

Commit

Permalink
Extract Dask executor to new daskexecutor provider
Browse files Browse the repository at this point in the history
  • Loading branch information
potiuk committed Jul 23, 2023
1 parent a4fdf72 commit e439272
Show file tree
Hide file tree
Showing 43 changed files with 1,028 additions and 450 deletions.
1 change: 1 addition & 0 deletions .github/ISSUE_TEMPLATE/airflow_providers_bug_report.yml
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,7 @@ body:
- cloudant
- cncf-kubernetes
- common-sql
- daskexecutor
- databricks
- datadog
- dbt-cloud
Expand Down
18 changes: 9 additions & 9 deletions CONTRIBUTING.rst
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
2 changes: 1 addition & 1 deletion Dockerfile
Original file line number Diff line number Diff line change
Expand Up @@ -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=""

Expand Down
2 changes: 1 addition & 1 deletion IMAGES.rst
Original file line number Diff line number Diff line change
Expand Up @@ -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::
Expand Down
18 changes: 9 additions & 9 deletions INSTALL
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
33 changes: 0 additions & 33 deletions airflow/config_templates/config.yml
Original file line number Diff line number Diff line change
Expand Up @@ -2010,39 +2010,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:
Expand Down
3 changes: 3 additions & 0 deletions airflow/executors/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -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__)
2 changes: 1 addition & 1 deletion airflow/executors/executor_loader.py
Original file line number Diff line number Diff line change
Expand Up @@ -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",
}
Expand Down
74 changes: 48 additions & 26 deletions airflow/providers/CREATING_COMMUNITY_PROVIDERS.rst
Original file line number Diff line number Diff line change
Expand Up @@ -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/
Expand All @@ -63,31 +66,39 @@ open a issue on GitHub so the community can help you.
│ ├── example_dags/
│ │ ├── __init__.py
│ │ └── example_<NEW_PROVIDER>.py
│ ├── executors/
│ │ ├── __init__.py
│ │ └── <NEW_PROVIDER>.py
│ ├── hooks/
│ │ ├── __init__.py
│ │ └── <NEW_PROVIDER>.py
│ ├── operators/
│ │ ├── __init__.py
│ │ └── <NEW_PROVIDER>.py
│ ├── sensors/
....
│ ├── transfers/
│ │ ├── __init__.py
│ │ └── <NEW_PROVIDER>.py
│ └── transfers/
│ └── triggers/
│ ├── __init__.py
│ └── <NEW_PROVIDER>.py
└── tests/providers/<NEW_PROVIDER>/
├── __init__.py
├── executors/
│ ├── __init__.py
│ └── test_<NEW_PROVIDER>.py
├── hooks/
│ ├── __init__.py
│ └── test_<NEW_PROVIDER>.py
├── operators/
│ ├── __init__.py
│ ├── test_<NEW_PROVIDER>.py
│ └── test_<NEW_PROVIDER>_system.py
├── sensors/
...
├── transfers/
│ ├── __init__.py
│ └── test_<NEW_PROVIDER>.py
└── transfers/
└── triggers/
├── __init__.py
└── test_<NEW_PROVIDER>.py
Expand All @@ -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/<NEW_PROVIDER>/
│ │ └── <NEW_PROVIDER>.png
│ └── apache-airflow-providers-<NEW_PROVIDER>/
│ ├── index.rst
│ ├── commits.rst
│ ├── connections.rst
│ └── operators/
│ └── <NEW_PROVIDER>.rst
└── providers/
└── <NEW_PROVIDER>/
├── provider.yaml
└── CHANGELOG.rst
├── INSTALL
├── CONTRIBUTING.rst
├── setup.py
├── airflow/
│ └── providers/
│ └── <NEW_PROVIDER>/
│ ├── provider.yaml
│ └── CHANGELOG.rst
└── docs/
├── spelling_wordlist.txt
├── apache-airflow/
│ └── extra-packages-ref.rst
├── integration-logos/<NEW_PROVIDER>/
│ └── <NEW_PROVIDER>.png
└── apache-airflow-providers-<NEW_PROVIDER>/
├── index.rst
├── commits.rst
├── connections.rst
└── operators/
└── <NEW_PROVIDER>.rst
Files automatically updated by pre-commit:
Expand Down Expand Up @@ -199,9 +211,19 @@ In the ``docs/apache-airflow-providers-<NEW_PROVIDER>/operators/<NEW_PROVIDER>.r
:end-before: [END howto_operator_<NEW_PROVIDER>]
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/<NEW_PROVIDER>/provider.yaml`` add information of your provider:

Expand Down
33 changes: 33 additions & 0 deletions airflow/providers/daskexecutor/CHANGELOG.rst
Original file line number Diff line number Diff line change
@@ -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.
41 changes: 41 additions & 0 deletions airflow/providers/daskexecutor/__init__.py
Original file line number Diff line number Diff line change
@@ -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
)
16 changes: 16 additions & 0 deletions airflow/providers/daskexecutor/executors/__init__.py
Original file line number Diff line number Diff line change
@@ -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.
File renamed without changes.
Loading

0 comments on commit e439272

Please sign in to comment.