Skip to content

Commit

Permalink
[AIRFLOW-8057] [AIP-31] Add @task decorator (#8962)
Browse files Browse the repository at this point in the history
Closes #8057. Closes #8056.
  • Loading branch information
casassg authored Jun 23, 2020
1 parent 7c587dc commit 23faab5
Show file tree
Hide file tree
Showing 6 changed files with 545 additions and 5 deletions.
18 changes: 18 additions & 0 deletions airflow/decorators.py
Original file line number Diff line number Diff line change
@@ -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.

from airflow.operators.python import task # noqa # pylint: disable=unused-import
5 changes: 5 additions & 0 deletions airflow/models/dag.py
Original file line number Diff line number Diff line change
Expand Up @@ -1318,6 +1318,11 @@ def get_downstream(task, level=0):
for t in self.roots:
get_downstream(t)

@property
def task(self):
from airflow.operators.python import task
return functools.partial(task, dag=self)

def add_task(self, task):
"""
Add a task to the DAG
Expand Down
145 changes: 144 additions & 1 deletion airflow/operators/python.py
Original file line number Diff line number Diff line change
Expand Up @@ -16,21 +16,25 @@
# specific language governing permissions and limitations
# under the License.

import functools
import inspect
import os
import pickle
import re
import sys
import types
from inspect import signature
from itertools import islice
from tempfile import TemporaryDirectory
from textwrap import dedent
from typing import Callable, Dict, Iterable, List, Optional
from typing import Any, Callable, Dict, Iterable, List, Optional, Tuple

import dill

from airflow.exceptions import AirflowException
from airflow.models import BaseOperator, SkipMixin
from airflow.models.dag import DAG, DagContext
from airflow.models.xcom_arg import XComArg
from airflow.utils.decorators import apply_defaults
from airflow.utils.process_utils import execute_in_subprocess
from airflow.utils.python_virtualenv import prepare_virtualenv
Expand Down Expand Up @@ -145,6 +149,145 @@ def execute_callable(self):
return self.python_callable(*self.op_args, **self.op_kwargs)


class _PythonFunctionalOperator(BaseOperator):
"""
Wraps a Python callable and captures args/kwargs when called for execution.
:param python_callable: A reference to an object that is callable
:type python_callable: python callable
:param op_kwargs: a dictionary of keyword arguments that will get unpacked
in your function (templated)
:type op_kwargs: dict
:param op_args: a list of positional arguments that will get unpacked when
calling your callable (templated)
:type op_args: list
:param multiple_outputs: if set, function return value will be
unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
Defaults to False.
:type multiple_outputs: bool
"""

template_fields = ('op_args', 'op_kwargs')
ui_color = PythonOperator.ui_color

# since we won't mutate the arguments, we should just do the shallow copy
# there are some cases we can't deepcopy the objects (e.g protobuf).
shallow_copy_attrs = ('python_callable',)

@apply_defaults
def __init__(
self,
python_callable: Callable,
task_id: str,
op_args: Tuple[Any],
op_kwargs: Dict[str, Any],
multiple_outputs: bool = False,
**kwargs
) -> None:
kwargs['task_id'] = self._get_unique_task_id(task_id, kwargs.get('dag', None))
super().__init__(**kwargs)
self.python_callable = python_callable

# Check that arguments can be binded
signature(python_callable).bind(*op_args, **op_kwargs)
self.multiple_outputs = multiple_outputs
self.op_args = op_args
self.op_kwargs = op_kwargs

@staticmethod
def _get_unique_task_id(task_id: str, dag: Optional[DAG] = None) -> str:
"""
Generate unique task id given a DAG (or if run in a DAG context)
Ids are generated by appending a unique number to the end of
the original task id.
Example:
task_id
task_id__1
task_id__2
...
task_id__20
"""
dag = dag or DagContext.get_current_dag()
if not dag or task_id not in dag.task_ids:
return task_id
core = re.split(r'__\d+$', task_id)[0]
suffixes = sorted(
[int(re.split(r'^.+__', task_id)[1])
for task_id in dag.task_ids
if re.match(rf'^{core}__\d+$', task_id)]
)
if not suffixes:
return f'{core}__1'
return f'{core}__{suffixes[-1] + 1}'

@staticmethod
def validate_python_callable(python_callable):
"""
Validate that python callable can be wrapped by operator.
Raises exception if invalid.
:param python_callable: Python object to be validated
:raises: TypeError, AirflowException
"""
if not callable(python_callable):
raise TypeError('`python_callable` param must be callable')
if 'self' in signature(python_callable).parameters.keys():
raise AirflowException('@task does not support methods')

def execute(self, context: Dict):
return_value = self.python_callable(*self.op_args, **self.op_kwargs)
self.log.debug("Done. Returned value was: %s", return_value)
if not self.multiple_outputs:
return return_value
if isinstance(return_value, dict):
for key in return_value.keys():
if not isinstance(key, str):
raise AirflowException('Returned dictionary keys must be strings when using '
f'multiple_outputs, found {key} ({type(key)}) instead')
for key, value in return_value.items():
self.xcom_push(context, key, value)
else:
raise AirflowException(f'Returned output was type {type(return_value)} expected dictionary '
'for multiple_outputs')
return return_value


def task(python_callable: Optional[Callable] = None, multiple_outputs: bool = False, **kwargs):
"""
Python operator decorator. Wraps a function into an Airflow operator.
Accepts kwargs for operator kwarg. Can be reused in a single DAG.
:param python_callable: Function to decorate
:type python_callable: Optional[Callable]
:param multiple_outputs: if set, function return value will be
unrolled to multiple XCom values. List/Tuples will unroll to xcom values
with index as key. Dict will unroll to xcom values with keys as XCom keys.
Defaults to False.
:type multiple_outputs: bool
"""
def wrapper(f):
"""
Python wrapper to generate PythonFunctionalOperator out of simple python functions.
Used for Airflow functional interface
"""
_PythonFunctionalOperator.validate_python_callable(f)
kwargs.setdefault('task_id', f.__name__)

@functools.wraps(f)
def factory(*args, **f_kwargs):
op = _PythonFunctionalOperator(python_callable=f, op_args=args, op_kwargs=f_kwargs,
multiple_outputs=multiple_outputs, **kwargs)
return XComArg(op)
return factory
if callable(python_callable):
return wrapper(python_callable)
elif python_callable is not None:
raise AirflowException('No args allowed while using @task, use kwargs instead')
return wrapper


class BranchPythonOperator(PythonOperator, SkipMixin):
"""
Allows a workflow to "branch" or follow a path following the execution
Expand Down
5 changes: 1 addition & 4 deletions airflow/ti_deps/deps/trigger_rule_dep.py
Original file line number Diff line number Diff line change
Expand Up @@ -18,10 +18,10 @@

from collections import Counter

import airflow
from airflow.ti_deps.deps.base_ti_dep import BaseTIDep
from airflow.utils.session import provide_session
from airflow.utils.state import State
from airflow.utils.trigger_rule import TriggerRule as TR


class TriggerRuleDep(BaseTIDep):
Expand Down Expand Up @@ -50,7 +50,6 @@ def _get_states_count_upstream_ti(ti, finished_tasks):

@provide_session
def _get_dep_statuses(self, ti, session, dep_context):
TR = airflow.utils.trigger_rule.TriggerRule
# Checking that all upstream dependencies have succeeded
if not ti.task.upstream_list:
yield self._passing_status(
Expand Down Expand Up @@ -111,8 +110,6 @@ def _evaluate_trigger_rule( # pylint: disable=too-many-branches
:type session: sqlalchemy.orm.session.Session
"""

TR = airflow.utils.trigger_rule.TriggerRule

task = ti.task
upstream = len(task.upstream_task_ids)
trigger_rule = task.trigger_rule
Expand Down
96 changes: 96 additions & 0 deletions docs/concepts.rst
Original file line number Diff line number Diff line change
Expand Up @@ -116,6 +116,46 @@ DAGs can be used as context managers to automatically assign new operators to th
op.dag is dag # True
.. _concepts:functional_dags:

Functional DAGs
---------------

DAGs can be defined using functional abstractions. Outputs and inputs are sent between tasks using
:ref:`XCom values <concepts:xcom>`. In addition, you can wrap functions as tasks using the
:ref:`task decorator <concepts:task_decorator>`. Airflow will also automatically add dependencies between
tasks to ensure that XCom messages are available when operators are executed.

Example DAG with functional abstraction

.. code-block:: python
with DAG(
'send_server_ip', default_args=default_args, schedule_interval=None
) as dag:
# Using default connection as it's set to httpbin.org by default
get_ip = SimpleHttpOperator(
task_id='get_ip', endpoint='get', method='GET', xcom_push=True
)
@dag.task(multiple_outputs=True)
def prepare_email(raw_json: str) -> Dict[str, str]:
external_ip = json.loads(raw_json)['origin']
return {
'subject':f'Server connected from {external_ip}',
'body': f'Seems like today your server executing Airflow is connected from the external IP {external_ip}<br>'
}
email_info = prepare_email(get_ip.output)
send_email = EmailOperator(
task_id='send_email',
to='[email protected]',
subject=email_info['subject'],
html_content=email_info['body']
)
.. _concepts:dagruns:

DAG Runs
Expand Down Expand Up @@ -173,6 +213,62 @@ Each task is a node in our DAG, and there is a dependency from task_1 to task_2:
We can say that task_1 is *upstream* of task_2, and conversely task_2 is *downstream* of task_1.
When a DAG Run is created, task_1 will start running and task_2 waits for task_1 to complete successfully before it may start.

.. _concepts:task_decorator:

Python task decorator
---------------------

Airflow ``task`` decorator converts any Python function to an Airflow operator.
The decorated function can be called once to set the arguments and key arguments for operator execution.


.. code-block:: python
with DAG('my_dag', start_date=datetime(2020, 5, 15)) as dag:
@dag.task
def hello_world():
print('hello world!')
# Also...
from airflow.decorators import task
@task
def hello_name(name: str):
print(f'hello {name}!')
hello_name('Airflow users')
Task decorator captures returned values and sends them to the :ref:`XCom backend <concepts:xcom>`. By default, returned
value is saved as a single XCom value. You can set ``multiple_outputs`` key argument to ``True`` to unroll dictionaries,
lists or tuples into seprate XCom values. This can be used with regular operators to create
:ref:`functional DAGs <concepts:functional_dags>`.

Calling a decorated function returns an ``XComArg`` instance. You can use it to set templated fields on downstream
operators.

You can call a decorated function more than once in a DAG. The decorated function will automatically generate
a unique ``task_id`` for each generated operator.

.. code-block:: python
with DAG('my_dag', start_date=datetime(2020, 5, 15)) as dag:
@dag.task
def update_user(user_id: int):
...
# Avoid generating this list dynamically to keep DAG topology stable between DAG runs
for user_id in user_ids:
update_user(user_id)
# This will generate an operator for each user_id
Task ids are generated by appending a number at the end of the original task id. For the above example, the DAG will have
the following task ids: ``[update_user, update_user__1, update_user__2, ... update_user__n]``.

Task Instances
==============

Expand Down
Loading

0 comments on commit 23faab5

Please sign in to comment.