Skip to content

Commit

Permalink
[AIRFLOW-6877] Add cross-provider dependencies as extras (#7506)
Browse files Browse the repository at this point in the history
Cross-provider dependencies are now extras in the backport
packages.
  • Loading branch information
potiuk committed Mar 7, 2020
1 parent cad20c2 commit 42eef38
Show file tree
Hide file tree
Showing 15 changed files with 578 additions and 125 deletions.
7 changes: 7 additions & 0 deletions .pre-commit-config.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -216,6 +216,13 @@ repos:
files: ^setup.cfg$
pass_filenames: false
require_serial: true
- id: build-providers-dependencies
name: Build cross-dependencies for providers packages
entry: "./scripts/ci/pre_commit_build_providers_dependencies.sh"
language: system
files: ^airflow/providers/.*\.py$|^tests/providers/.*\.py$
pass_filenames: false
require_serial: true
- id: pydevd
language: pygrep
name: Check for pydevd debug statements accidentally left
Expand Down
57 changes: 57 additions & 0 deletions CONTRIBUTING.rst
Original file line number Diff line number Diff line change
Expand Up @@ -316,6 +316,63 @@ Limitations:
They are optimized for repeatability of tests, maintainability and speed of building rather
than production performance. The production images are not yet officially published.

Backport providers packages
---------------------------

Since we are developing new operators in the master branch, we prepared backport packages ready to be
installed for Airflow 1.10.* series. Those backport operators (the tested ones) are going to be released
in PyPi and we are going to maintain the list at
`Backported providers package page <https://cwiki.apache.org/confluence/display/AIRFLOW/Backported+providers+packages+for+Airflow+1.10.*+series>`_

Some of the packages have cross-dependencies with other providers packages. This typically happens for
transfer operators where operators use hooks from the other providers in case they are transferring
data between the providers. The list of dependencies is maintained (automatically with pre-commits)
in the ``airflow/providers/dependencies.json``. Pre-commits are also used to generate dependencies.
The dependency list is automatically used during pypi packages generation.

Cross-dependencies between provider packages are converted into extras - if you need functionality from
the other provider package you can install it adding [extra] after the apache-airflow-providers-PROVIDER
for example ``pip install apache-airflow-providers-google[amazon]`` in case you want to use GCP's
transfer operators from Amazon ECS.

If you add a new dependency between different providers packages, it will be detected automatically during
pre-commit phase and pre-commit will fail - and add entry in dependencies.json so that the package extra
dependencies are properly added when package is installed.

You can regenerate the whole list of provider dependencies by running this command (you need to have
``pre-commits`` installed).

.. code-block:: bash
pre-commit run build-providers-dependencies
Here is the list of packages and their extras:


.. START PACKAGE DEPENDENCIES HERE
========================== ===========================
Package Extras
========================== ===========================
amazon apache.hive,google,imap,mongo,postgres,ssh
apache.druid apache.hive
apache.hive amazon,microsoft.mssql,mysql,presto,samba,vertica
apache.livy http
dingding http
discord http
google amazon,apache.cassandra,cncf.kubernetes,microsoft.azure,microsoft.mssql,mysql,postgres,sftp
microsoft.azure oracle
microsoft.mssql odbc
mysql amazon,presto,vertica
opsgenie http
postgres amazon
sftp ssh
slack http
========================== ===========================

.. END PACKAGE DEPENDENCIES HERE
Static code checks
==================

Expand Down
3 changes: 3 additions & 0 deletions README.md
Original file line number Diff line number Diff line change
Expand Up @@ -132,6 +132,9 @@ environments to Airflow 2.0.
More information about the status and releases of the back-ported packages are available
at [Backported providers package page](https://cwiki.apache.org/confluence/display/AIRFLOW/Backported+providers+packages+for+Airflow+1.10.*+series)

Dependencies between packages are stored in ``airflow/providers/dependencies.json``. See
[CONTRIBUTING.rst](https://github.com/apache/airflow/blob/master/CONTRIBUTING.rst#backport-providers-packages)

## Contributing

Want to help build Apache Airflow? Check out our [contributing documentation](https://github.com/apache/airflow/blob/master/CONTRIBUTING.rst).
Expand Down
1 change: 0 additions & 1 deletion airflow/providers/amazon/aws/operators/sftp_to_s3.py
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.

from tempfile import NamedTemporaryFile
from urllib.parse import urlparse

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,50 +21,42 @@
import logging

from airflow import DAG
from airflow.providers.cncf.kubernetes.operators.kubernetes_pod import KubernetesPodOperator
from airflow.utils.dates import days_ago

log = logging.getLogger(__name__)

try:
# Kubernetes is optional, so not available in vanilla Airflow
# pip install 'apache-airflow[kubernetes]'
from airflow.providers.cncf.kubernetes.operators.kubernetes_pod import KubernetesPodOperator

default_args = {
'owner': 'airflow',
'start_date': days_ago(2)
}

with DAG(
dag_id='example_kubernetes_operator',
default_args=default_args,
schedule_interval=None,
tags=['example'],
) as dag:

tolerations = [
{
'key': "key",
'operator': 'Equal',
'value': 'value'
}
]

k = KubernetesPodOperator(
namespace='default',
image="ubuntu:16.04",
cmds=["bash", "-cx"],
arguments=["echo", "10"],
labels={"foo": "bar"},
name="airflow-test-pod",
in_cluster=False,
task_id="task",
get_logs=True,
is_delete_operator_pod=False,
tolerations=tolerations
)

except ImportError as e:
log.warning("Could not import KubernetesPodOperator: %s, ", str(e))
log.warning("Install kubernetes dependencies with: \n"
" pip install 'apache-airflow[kubernetes]'")
default_args = {
'owner': 'airflow',
'start_date': days_ago(2)
}

with DAG(
dag_id='example_kubernetes_operator',
default_args=default_args,
schedule_interval=None,
tags=['example'],
) as dag:

tolerations = [
{
'key': "key",
'operator': 'Equal',
'value': 'value'
}
]

k = KubernetesPodOperator(
namespace='default',
image="ubuntu:16.04",
cmds=["bash", "-cx"],
arguments=["echo", "10"],
labels={"foo": "bar"},
name="airflow-test-pod",
in_cluster=False,
task_id="task",
get_logs=True,
is_delete_operator_pod=False,
tolerations=tolerations
)
63 changes: 63 additions & 0 deletions airflow/providers/dependencies.json
Original file line number Diff line number Diff line change
@@ -0,0 +1,63 @@
{
"amazon": [
"apache.hive",
"google",
"imap",
"mongo",
"postgres",
"ssh"
],
"apache.druid": [
"apache.hive"
],
"apache.hive": [
"amazon",
"microsoft.mssql",
"mysql",
"presto",
"samba",
"vertica"
],
"apache.livy": [
"http"
],
"dingding": [
"http"
],
"discord": [
"http"
],
"google": [
"amazon",
"apache.cassandra",
"cncf.kubernetes",
"microsoft.azure",
"microsoft.mssql",
"mysql",
"postgres",
"sftp"
],
"microsoft.azure": [
"oracle"
],
"microsoft.mssql": [
"odbc"
],
"mysql": [
"amazon",
"presto",
"vertica"
],
"opsgenie": [
"http"
],
"postgres": [
"amazon"
],
"sftp": [
"ssh"
],
"slack": [
"http"
]
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,13 +19,13 @@
"""
This module contains Google Cloud Transfer operators.
"""

from copy import deepcopy
from datetime import date, time
from typing import Dict, Optional

from airflow.exceptions import AirflowException
from airflow.models import BaseOperator
from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
from airflow.providers.google.cloud.hooks.cloud_storage_transfer_service import (
ACCESS_KEY_ID, AWS_ACCESS_KEY, AWS_S3_DATA_SOURCE, BUCKET_NAME, DAY, DESCRIPTION, GCS_DATA_SINK,
GCS_DATA_SOURCE, HOURS, HTTP_DATA_SOURCE, MINUTES, MONTH, OBJECT_CONDITIONS, PROJECT_ID, SCHEDULE,
Expand All @@ -34,11 +34,6 @@
)
from airflow.utils.decorators import apply_defaults

try:
from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
except ImportError: # pragma: no cover
AwsBaseHook = None # type: ignore


class TransferJobPreprocessor:
"""
Expand Down
4 changes: 3 additions & 1 deletion airflow/providers/sftp/operators/sftp.py
Original file line number Diff line number Diff line change
Expand Up @@ -26,8 +26,10 @@
from airflow.utils.decorators import apply_defaults


# pylint: disable=missing-docstring
class SFTPOperation:
"""
Operation that can be used with SFTP/
"""
PUT = 'put'
GET = 'get'

Expand Down
16 changes: 16 additions & 0 deletions backport_packages/__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.
Loading

0 comments on commit 42eef38

Please sign in to comment.