Skip to content

Commit a9772cf

Browse files
dimbermanashb
andauthored
Add a Docker Taskflow decorator (#15330)
Add the ability to run @task.docker on a python function and turn it into a DockerOperator that can run that python function remotely. ``` @task.docker( image="quay.io/bitnami/python:3.8.8", force_pull=True, docker_url="unix://var/run/docker.sock", network_mode="bridge", api_version='auto', ) def f(): import random return [random.random() for i in range(10000000)] ``` One notable aspect of this architecture is that we had to build it to make as few assumptions about user setups as possible. We could not share a volume between the worker and the container as this would break if the user runs the airflow worker on a docker container. We could not assume that users would have any specialized system libraries on their images (this implementation only requires python 3 and bash). To work with these requirements, we use base64 encoding to store a jinja generated python file and inputs (which are generated using the same functions used by the PythonVirtualEnvOperator). Once the container starts, it uses these environment variables to deserialize the strings, run the function, and store the result in a file located at /tmp/script.out. Once the function completes, we create a sleep loop until the DockerOperator retrieves the result via docker's get_archive API. This result can then be deserialized using pickle and sent to Airflow's XCom library in the same fashion as a python or python_virtualenv result. Co-authored-by: Ash Berlin-Taylor <ash_github@firemirror.com> Co-authored-by: Ash Berlin-Taylor <ash@apache.org>
1 parent bada372 commit a9772cf

18 files changed

+845
-175
lines changed

airflow/decorators/__init__.py

+22-115
Original file line numberDiff line numberDiff line change
@@ -15,129 +15,36 @@
1515
# specific language governing permissions and limitations
1616
# under the License.
1717

18-
from typing import Callable, Dict, Iterable, List, Optional, Union
18+
from typing import TYPE_CHECKING
1919

20-
from airflow.decorators.python import python_task
21-
from airflow.decorators.python_virtualenv import _virtualenv_task
20+
from airflow.decorators.python import PythonDecoratorMixin, python_task # noqa
21+
from airflow.decorators.python_virtualenv import PythonVirtualenvDecoratorMixin
2222
from airflow.decorators.task_group import task_group # noqa
2323
from airflow.models.dag import dag # noqa
24+
from airflow.providers_manager import ProvidersManager
2425

2526

26-
class _TaskDecorator:
27-
def __call__(
28-
self, python_callable: Optional[Callable] = None, multiple_outputs: Optional[bool] = None, **kwargs
29-
):
30-
"""
31-
Python operator decorator. Wraps a function into an Airflow operator.
32-
Accepts kwargs for operator kwarg. This decorator can be reused in a single DAG.
27+
class _TaskDecorator(PythonDecoratorMixin, PythonVirtualenvDecoratorMixin):
28+
def __getattr__(self, name):
29+
if name.startswith("__"):
30+
raise AttributeError(f'{type(self).__name__} has no attribute {name!r}')
31+
decorators = ProvidersManager().taskflow_decorators
32+
if name not in decorators:
33+
raise AttributeError(f"task decorator {name!r} not found")
34+
return decorators[name]
3335

34-
:param python_callable: Function to decorate
35-
:type python_callable: Optional[Callable]
36-
:param multiple_outputs: if set, function return value will be
37-
unrolled to multiple XCom values. List/Tuples will unroll to xcom values
38-
with index as key. Dict will unroll to xcom values with keys as XCom keys.
39-
Defaults to False.
40-
:type multiple_outputs: bool
41-
"""
42-
return self.python(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs)
4336

44-
@staticmethod
45-
def python(python_callable: Optional[Callable] = None, multiple_outputs: Optional[bool] = None, **kwargs):
46-
"""
47-
Python operator decorator. Wraps a function into an Airflow operator.
48-
Accepts kwargs for operator kwarg. This decorator can be reused in a single DAG.
37+
# [START mixin_for_autocomplete]
38+
if TYPE_CHECKING:
39+
try:
40+
from airflow.providers.docker.decorators.docker import DockerDecoratorMixin
4941

50-
:param python_callable: Function to decorate
51-
:type python_callable: Optional[Callable]
52-
:param multiple_outputs: if set, function return value will be
53-
unrolled to multiple XCom values. List/Tuples will unroll to xcom values
54-
with index as key. Dict will unroll to xcom values with keys as XCom keys.
55-
Defaults to False.
56-
:type multiple_outputs: bool
57-
"""
58-
return python_task(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs)
59-
60-
@staticmethod
61-
def virtualenv(
62-
python_callable: Optional[Callable] = None,
63-
multiple_outputs: Optional[bool] = None,
64-
requirements: Optional[Iterable[str]] = None,
65-
python_version: Optional[Union[str, int, float]] = None,
66-
use_dill: bool = False,
67-
system_site_packages: bool = True,
68-
string_args: Optional[Iterable[str]] = None,
69-
templates_dict: Optional[Dict] = None,
70-
templates_exts: Optional[List[str]] = None,
71-
**kwargs,
72-
):
73-
"""
74-
Allows one to run a function in a virtualenv that is
75-
created and destroyed automatically (with certain caveats).
76-
77-
The function must be defined using def, and not be
78-
part of a class. All imports must happen inside the function
79-
and no variables outside of the scope may be referenced. A global scope
80-
variable named virtualenv_string_args will be available (populated by
81-
string_args). In addition, one can pass stuff through op_args and op_kwargs, and one
82-
can use a return value.
83-
Note that if your virtualenv runs in a different Python major version than Airflow,
84-
you cannot use return values, op_args, op_kwargs, or use any macros that are being provided to
85-
Airflow through plugins. You can use string_args though.
86-
87-
.. seealso::
88-
For more information on how to use this operator, take a look at the guide:
89-
:ref:`howto/operator:PythonVirtualenvOperator`
90-
91-
:param python_callable: A python function with no references to outside variables,
92-
defined with def, which will be run in a virtualenv
93-
:type python_callable: function
94-
:param multiple_outputs: if set, function return value will be
95-
unrolled to multiple XCom values. List/Tuples will unroll to xcom values
96-
with index as key. Dict will unroll to xcom values with keys as XCom keys.
97-
Defaults to False.
98-
:type multiple_outputs: bool
99-
:param requirements: A list of requirements as specified in a pip install command
100-
:type requirements: list[str]
101-
:param python_version: The Python version to run the virtualenv with. Note that
102-
both 2 and 2.7 are acceptable forms.
103-
:type python_version: Optional[Union[str, int, float]]
104-
:param use_dill: Whether to use dill to serialize
105-
the args and result (pickle is default). This allow more complex types
106-
but requires you to include dill in your requirements.
107-
:type use_dill: bool
108-
:param system_site_packages: Whether to include
109-
system_site_packages in your virtualenv.
110-
See virtualenv documentation for more information.
111-
:type system_site_packages: bool
112-
:param op_args: A list of positional arguments to pass to python_callable.
113-
:type op_args: list
114-
:param op_kwargs: A dict of keyword arguments to pass to python_callable.
115-
:type op_kwargs: dict
116-
:param string_args: Strings that are present in the global var virtualenv_string_args,
117-
available to python_callable at runtime as a list[str]. Note that args are split
118-
by newline.
119-
:type string_args: list[str]
120-
:param templates_dict: a dictionary where the values are templates that
121-
will get templated by the Airflow engine sometime between
122-
``__init__`` and ``execute`` takes place and are made available
123-
in your callable's context after the template has been applied
124-
:type templates_dict: dict of str
125-
:param templates_exts: a list of file extensions to resolve while
126-
processing templated fields, for examples ``['.sql', '.hql']``
127-
:type templates_exts: list[str]
128-
"""
129-
return _virtualenv_task(
130-
python_callable=python_callable,
131-
multiple_outputs=multiple_outputs,
132-
requirements=requirements,
133-
python_version=python_version,
134-
use_dill=use_dill,
135-
system_site_packages=system_site_packages,
136-
string_args=string_args,
137-
templates_dict=templates_dict,
138-
templates_exts=templates_exts,
139-
**kwargs,
140-
)
42+
class _DockerTask(_TaskDecorator, DockerDecoratorMixin):
43+
pass
14144

45+
_TaskDecorator = _DockerTask
46+
except ImportError:
47+
pass
48+
# [END mixin_for_autocomplete]
14249

14350
task = _TaskDecorator()

airflow/decorators/base.py

+1-2
Original file line numberDiff line numberDiff line change
@@ -132,8 +132,7 @@ def __init__(
132132

133133
def execute(self, context: Dict):
134134
return_value = super().execute(context)
135-
self._handle_output(return_value=return_value, context=context, xcom_push=self.xcom_push)
136-
return return_value
135+
return self._handle_output(return_value=return_value, context=context, xcom_push=self.xcom_push)
137136

138137
def _handle_output(self, return_value: Any, context: Dict, xcom_push: Callable):
139138
"""

airflow/decorators/python.py

+39-1
Original file line numberDiff line numberDiff line change
@@ -62,11 +62,49 @@ def __init__(
6262
T = TypeVar("T", bound=Callable)
6363

6464

65+
class PythonDecoratorMixin:
66+
"""
67+
Helper class for inheritance. This class is only used for the __init__.pyi so that IDEs
68+
will autocomplete docker decorator functions
69+
70+
:meta private:
71+
"""
72+
73+
def __call__(
74+
self, python_callable: Optional[Callable] = None, multiple_outputs: Optional[bool] = None, **kwargs
75+
):
76+
return self.python(python_callable, multiple_outputs, **kwargs)
77+
78+
def python(
79+
self, python_callable: Optional[Callable] = None, multiple_outputs: Optional[bool] = None, **kwargs
80+
):
81+
"""
82+
Python operator decorator. Wraps a function into an Airflow operator.
83+
84+
Accepts kwargs for operator kwarg. Can be reused in a single DAG.
85+
86+
:param python_callable: Function to decorate
87+
:type python_callable: Optional[Callable]
88+
:param multiple_outputs: if set, function return value will be
89+
unrolled to multiple XCom values. List/Tuples will unroll to xcom values
90+
with index as key. Dict will unroll to xcom values with keys as XCom keys.
91+
Defaults to False.
92+
:type multiple_outputs: bool
93+
"""
94+
return task_decorator_factory(
95+
python_callable=python_callable,
96+
multiple_outputs=multiple_outputs,
97+
decorated_operator_class=_PythonDecoratedOperator,
98+
**kwargs,
99+
)
100+
101+
65102
def python_task(
66103
python_callable: Optional[Callable] = None, multiple_outputs: Optional[bool] = None, **kwargs
67104
):
68105
"""
69-
Python operator decorator. Wraps a function into an Airflow operator.
106+
Wraps a function into an Airflow operator.
107+
70108
Accepts kwargs for operator kwarg. Can be reused in a single DAG.
71109
72110
:param python_callable: Function to decorate

airflow/decorators/python_virtualenv.py

+27-18
Original file line numberDiff line numberDiff line change
@@ -70,24 +70,33 @@ def get_python_source(self):
7070
T = TypeVar("T", bound=Callable)
7171

7272

73-
def _virtualenv_task(
74-
python_callable: Optional[Callable] = None, multiple_outputs: Optional[bool] = None, **kwargs
75-
):
73+
class PythonVirtualenvDecoratorMixin:
7674
"""
77-
Python operator decorator. Wraps a function into an Airflow operator.
78-
Accepts kwargs for operator kwarg. Can be reused in a single DAG.
75+
Helper class for inheritance. This class is only used for the __init__.pyi so that IDEs
76+
will autocomplete docker decorator functions
7977
80-
:param python_callable: Function to decorate
81-
:type python_callable: Optional[Callable]
82-
:param multiple_outputs: if set, function return value will be
83-
unrolled to multiple XCom values. List/Tuples will unroll to xcom values
84-
with index as key. Dict will unroll to xcom values with keys as XCom keys.
85-
Defaults to False.
86-
:type multiple_outputs: bool
78+
:meta private:
8779
"""
88-
return task_decorator_factory(
89-
python_callable=python_callable,
90-
multiple_outputs=multiple_outputs,
91-
decorated_operator_class=_PythonVirtualenvDecoratedOperator,
92-
**kwargs,
93-
)
80+
81+
def virtualenv(
82+
self, python_callable: Optional[Callable] = None, multiple_outputs: Optional[bool] = None, **kwargs
83+
):
84+
"""
85+
Wraps a python function into an Airflow operator to run via a Python virtual environment.
86+
87+
Accepts kwargs for operator kwarg. Can be reused in a single DAG.
88+
89+
:param python_callable: Function to decorate
90+
:type python_callable: Optional[Callable]
91+
:param multiple_outputs: if set, function return value will be
92+
unrolled to multiple XCom values. List/Tuples will unroll to xcom values
93+
with index as key. Dict will unroll to xcom values with keys as XCom keys.
94+
Defaults to False.
95+
:type multiple_outputs: bool
96+
"""
97+
return task_decorator_factory(
98+
python_callable=python_callable,
99+
multiple_outputs=multiple_outputs,
100+
decorated_operator_class=_PythonVirtualenvDecoratedOperator,
101+
**kwargs,
102+
)

airflow/example_dags/tutorial_taskflow_api_etl_virtualenv.py airflow/example_dags/tutorial_taskflow_api_etl_docker_virtualenv.py

+1-1
Original file line numberDiff line numberDiff line change
@@ -62,7 +62,7 @@ def extract():
6262
# [END extract_virtualenv]
6363

6464
# [START transform_docker]
65-
@task.virtualenv(multiple_outputs=True)
65+
@task.docker(image='python:3.9-slim-buster', multiple_outputs=True)
6666
def transform(order_data_dict: dict):
6767
"""
6868
#### Transform task

airflow/provider.yaml.schema.json

+12
Original file line numberDiff line numberDiff line change
@@ -233,6 +233,18 @@
233233
"type": "object",
234234
"description": "Additional extras that the provider should have"
235235
},
236+
"task-decorators": {
237+
"type": "array",
238+
"description": "Decorators to use with the TaskFlow API. Can be accessed by users via '@task.<name>'",
239+
"items": {
240+
"name": {
241+
"type": "string"
242+
},
243+
"path": {
244+
"type": "string"
245+
}
246+
}
247+
},
236248
"secrets-backends": {
237249
"type": "array",
238250
"description": "Secrets Backend class names",

airflow/provider_info.schema.json

+12
Original file line numberDiff line numberDiff line change
@@ -70,6 +70,18 @@
7070
"items": {
7171
"type": "string"
7272
}
73+
},
74+
"task-decorators": {
75+
"type": "array",
76+
"description": "Apply custom decorators to the TaskFlow API. Can be accessed by users via '@task.<name>'",
77+
"items": {
78+
"name": {
79+
"type": "string"
80+
},
81+
"path": {
82+
"type": "string"
83+
}
84+
}
7385
}
7486
},
7587
"required": [
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,17 @@
1+
#
2+
# Licensed to the Apache Software Foundation (ASF) under one
3+
# or more contributor license agreements. See the NOTICE file
4+
# distributed with this work for additional information
5+
# regarding copyright ownership. The ASF licenses this file
6+
# to you under the Apache License, Version 2.0 (the
7+
# "License"); you may not use this file except in compliance
8+
# with the License. You may obtain a copy of the License at
9+
#
10+
# http://www.apache.org/licenses/LICENSE-2.0
11+
#
12+
# Unless required by applicable law or agreed to in writing,
13+
# software distributed under the License is distributed on an
14+
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15+
# KIND, either express or implied. See the License for the
16+
# specific language governing permissions and limitations
17+
# under the License.

0 commit comments

Comments
 (0)