Skip to content

Commit

Permalink
Implement ExternalPythonOperator (#25780)
Browse files Browse the repository at this point in the history
This Operator works very similarly to PythonVirtualenvOperator - but
instead of creating a virtualenv dynamically, it expects the
env to be available in the environment that Airlfow is run in.

The PR adds not only the implemenat the operator, but also
documents the operator's use and adds best-practices chapter
that explains the differences between different ways how you can
achieve separation of dependencies between different tasks. This
has been a question added many times in by our users, so adding
this operator and outlining future aspects of AIP-46 and AIP-43
that will make separate docker images another option is also
part of this change.
  • Loading branch information
potiuk committed Sep 6, 2022
1 parent 6931fbf commit 55928b9
Show file tree
Hide file tree
Showing 19 changed files with 1,110 additions and 187 deletions.
3 changes: 3 additions & 0 deletions airflow/decorators/__init__.py
Expand Up @@ -19,6 +19,7 @@

from airflow.decorators.base import TaskDecorator
from airflow.decorators.branch_python import branch_task
from airflow.decorators.external_python import external_python_task
from airflow.decorators.python import python_task
from airflow.decorators.python_virtualenv import virtualenv_task
from airflow.decorators.task_group import task_group
Expand All @@ -34,6 +35,7 @@
"task_group",
"python_task",
"virtualenv_task",
"external_python_task",
"branch_task",
]

Expand All @@ -43,6 +45,7 @@ class TaskDecoratorCollection:

python = staticmethod(python_task)
virtualenv = staticmethod(virtualenv_task)
external_python = staticmethod(external_python_task)
branch = staticmethod(branch_task)

__call__: Any = python # Alias '@task' to '@task.python'.
Expand Down
33 changes: 33 additions & 0 deletions airflow/decorators/__init__.pyi
Expand Up @@ -26,6 +26,7 @@ from kubernetes.client import models as k8s

from airflow.decorators.base import FParams, FReturn, Task, TaskDecorator
from airflow.decorators.branch_python import branch_task
from airflow.decorators.external_python import external_python_task
from airflow.decorators.python import python_task
from airflow.decorators.python_virtualenv import virtualenv_task
from airflow.decorators.task_group import task_group
Expand All @@ -41,6 +42,7 @@ __all__ = [
"task_group",
"python_task",
"virtualenv_task",
"external_python_task",
"branch_task",
]

Expand Down Expand Up @@ -126,6 +128,37 @@ class TaskDecoratorCollection:
"""
@overload
def virtualenv(self, python_callable: Callable[FParams, FReturn]) -> Task[FParams, FReturn]: ...
def external_python(
self,
*,
python: str,
multiple_outputs: Optional[bool] = None,
# 'python_callable', 'op_args' and 'op_kwargs' since they are filled by
# _PythonVirtualenvDecoratedOperator.
use_dill: bool = False,
templates_dict: Optional[Mapping[str, Any]] = None,
show_return_value_in_logs: bool = True,
**kwargs,
) -> TaskDecorator:
"""Create a decorator to convert the decorated callable to a virtual environment task.
:param python: Full path string (file-system specific) that points to a Python binary inside
a virtualenv that should be used (in ``VENV/bin`` folder). Should be absolute path
(so usually start with "/" or "X:/" depending on the filesystem/os used).
:param multiple_outputs: If set, function return value will be unrolled to multiple XCom values.
Dict will unroll to XCom values with keys as XCom keys. Defaults to False.
:param use_dill: Whether to use dill to serialize
the args and result (pickle is default). This allow more complex types
but requires you to include dill in your requirements.
:param templates_dict: a dictionary where the values are templates that
will get templated by the Airflow engine sometime between
``__init__`` and ``execute`` takes place and are made available
in your callable's context after the template has been applied.
:param show_return_value_in_logs: a bool value whether to show return_value
logs. Defaults to True, which allows return value log output.
It can be set to False to prevent log output of return value when you return huge data
such as transmission a large amount of XCom to TaskAPI.
"""
@overload
def branch(self, *, multiple_outputs: Optional[bool] = None, **kwargs) -> TaskDecorator:
"""Create a decorator to wrap the decorated callable into a BranchPythonOperator.
Expand Down
2 changes: 1 addition & 1 deletion airflow/decorators/branch_python.py
Expand Up @@ -21,7 +21,7 @@

from airflow.decorators.base import DecoratedOperator, TaskDecorator, task_decorator_factory
from airflow.operators.python import BranchPythonOperator
from airflow.utils.python_virtualenv import remove_task_decorator
from airflow.utils.decorators import remove_task_decorator


class _BranchPythonDecoratedOperator(DecoratedOperator, BranchPythonOperator):
Expand Down
101 changes: 101 additions & 0 deletions airflow/decorators/external_python.py
@@ -0,0 +1,101 @@
# 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.

import inspect
from textwrap import dedent
from typing import Callable, Optional, Sequence

from airflow.decorators.base import DecoratedOperator, TaskDecorator, task_decorator_factory
from airflow.operators.python import ExternalPythonOperator
from airflow.utils.decorators import remove_task_decorator


class _PythonExternalDecoratedOperator(DecoratedOperator, ExternalPythonOperator):
"""
Wraps a Python callable and captures args/kwargs when called for execution.
:param python: Full path string (file-system specific) that points to a Python binary inside
a virtualenv that should be used (in ``VENV/bin`` folder). Should be absolute path
(so usually start with "/" or "X:/" depending on the filesystem/os used).
:param python_callable: A reference to an object that is callable
:param op_kwargs: a dictionary of keyword arguments that will get unpacked
in your function (templated)
:param op_args: a list of positional arguments that will get unpacked when
calling your callable (templated)
:param multiple_outputs: If set to True, the decorated function's return value will be unrolled to
multiple XCom values. Dict will unroll to XCom values with its keys as XCom keys. Defaults to False.
"""

template_fields: Sequence[str] = ('op_args', 'op_kwargs')
template_fields_renderers = {"op_args": "py", "op_kwargs": "py"}

# 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: Sequence[str] = ('python_callable',)

custom_operator_name: str = '@task.external_python'

def __init__(self, *, python_callable, op_args, op_kwargs, **kwargs) -> None:
kwargs_to_upstream = {
"python_callable": python_callable,
"op_args": op_args,
"op_kwargs": op_kwargs,
}
super().__init__(
kwargs_to_upstream=kwargs_to_upstream,
python_callable=python_callable,
op_args=op_args,
op_kwargs=op_kwargs,
**kwargs,
)

def get_python_source(self):
raw_source = inspect.getsource(self.python_callable)
res = dedent(raw_source)
res = remove_task_decorator(res, "@task.external_python")
return res


def external_python_task(
python: Optional[str] = None,
python_callable: Optional[Callable] = None,
multiple_outputs: Optional[bool] = None,
**kwargs,
) -> TaskDecorator:
"""Wraps a callable into an Airflow operator to run via a Python virtual environment.
Accepts kwargs for operator kwarg. Can be reused in a single DAG.
This function is only used during type checking or auto-completion.
:meta private:
:param python: Full path string (file-system specific) that points to a Python binary inside
a virtualenv that should be used (in ``VENV/bin`` folder). Should be absolute path
(so usually start with "/" or "X:/" depending on the filesystem/os used).
:param python_callable: Function to decorate
:param multiple_outputs: If set to True, the decorated function's return value will be unrolled to
multiple XCom values. Dict will unroll to XCom values with its keys as XCom keys.
Defaults to False.
"""
return task_decorator_factory(
python=python,
python_callable=python_callable,
multiple_outputs=multiple_outputs,
decorated_operator_class=_PythonExternalDecoratedOperator,
**kwargs,
)
2 changes: 1 addition & 1 deletion airflow/decorators/python_virtualenv.py
Expand Up @@ -21,7 +21,7 @@

from airflow.decorators.base import DecoratedOperator, TaskDecorator, task_decorator_factory
from airflow.operators.python import PythonVirtualenvOperator
from airflow.utils.python_virtualenv import remove_task_decorator
from airflow.utils.decorators import remove_task_decorator


class _PythonVirtualenvDecoratedOperator(DecoratedOperator, PythonVirtualenvOperator):
Expand Down
37 changes: 35 additions & 2 deletions airflow/example_dags/example_python_operator.py
Expand Up @@ -21,7 +21,10 @@
virtual environment.
"""
import logging
import os
import shutil
import sys
import tempfile
import time
from pprint import pprint

Expand All @@ -32,6 +35,10 @@

log = logging.getLogger(__name__)

PYTHON = sys.executable

BASE_DIR = tempfile.gettempdir()

with DAG(
dag_id='example_python_operator',
schedule=None,
Expand Down Expand Up @@ -86,10 +93,36 @@ def callable_virtualenv():
print(Back.GREEN + 'and with a green background')
print(Style.DIM + 'and in dim text')
print(Style.RESET_ALL)
for _ in range(10):
for _ in range(4):
print(Style.DIM + 'Please wait...', flush=True)
sleep(10)
sleep(1)
print('Finished')

virtualenv_task = callable_virtualenv()
# [END howto_operator_python_venv]

sleeping_task >> virtualenv_task

# [START howto_operator_external_python]
@task.external_python(task_id="external_python", python=os.fspath(sys.executable))
def callable_external_python():
"""
Example function that will be performed in a virtual environment.
Importing at the module level ensures that it will not attempt to import the
library before it is installed.
"""
import sys
from time import sleep

print(f"Running task via {sys.executable}")
print("Sleeping")
for _ in range(4):
print('Please wait...', flush=True)
sleep(1)
print('Finished')

external_python_task = callable_external_python()
# [END howto_operator_external_python]

run_this >> external_python_task

0 comments on commit 55928b9

Please sign in to comment.