airflow.operators.python

Module Contents

Classes

PythonOperator

Executes a Python callable

BranchPythonOperator

Allows a workflow to "branch" or follow a path following the execution

ShortCircuitOperator

Allows a pipeline to continue based on the result of a python_callable.

PythonVirtualenvOperator

Allows one to run a function in a virtualenv that is created and destroyed

ExternalPythonOperator

Allows one to run a function in a virtualenv that is not re-created but used as is

Functions

task([python_callable, multiple_outputs])

Deprecated function that calls @task.python and allows users to turn a python function into

get_current_context()

Obtain the execution context for the currently executing operator without

airflow.operators.python.task(python_callable=None, multiple_outputs=None, **kwargs)[source]

Deprecated function that calls @task.python and allows users to turn a python function into an Airflow task. Please use the following instead:

from airflow.decorators import task

@task def my_task()

Parameters
  • python_callable (Callable | None) – A reference to an object that is callable

  • op_kwargs – a dictionary of keyword arguments that will get unpacked in your function (templated)

  • op_args – a list of positional arguments that will get unpacked when calling your callable (templated)

  • multiple_outputs (bool | None) – 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.

Returns

class airflow.operators.python.PythonOperator(*, python_callable, op_args=None, op_kwargs=None, templates_dict=None, templates_exts=None, show_return_value_in_logs=True, **kwargs)[source]

Bases: airflow.models.baseoperator.BaseOperator

Executes a Python callable

See also

For more information on how to use this operator, take a look at the guide: PythonOperator

When running your callable, Airflow will pass a set of keyword arguments that can be used in your function. This set of kwargs correspond exactly to what you can use in your jinja templates. For this to work, you need to define **kwargs in your function header, or you can add directly the keyword arguments you would like to get - for example with the below code your callable will get the values of ti and next_ds context variables.

With explicit arguments:

def my_python_callable(ti, next_ds):
    pass

With kwargs:

def my_python_callable(**kwargs):
    ti = kwargs["ti"]
    next_ds = kwargs["next_ds"]
Parameters
  • python_callable (Callable) – A reference to an object that is callable

  • op_kwargs (Mapping[str, Any] | None) – a dictionary of keyword arguments that will get unpacked in your function

  • op_args (Collection[Any] | None) – a list of positional arguments that will get unpacked when calling your callable

  • templates_dict (dict[str, Any] | None) – 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. (templated)

  • templates_exts (Sequence[str] | None) – a list of file extensions to resolve while processing templated fields, for examples ['.sql', '.hql']

  • show_return_value_in_logs (bool) – 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.

template_fields: Sequence[str] = ('templates_dict', 'op_args', 'op_kwargs')[source]
template_fields_renderers[source]
BLUE = '#ffefeb'[source]
ui_color[source]
shallow_copy_attrs: Sequence[str] = ('python_callable', 'op_kwargs')[source]
execute(context)[source]

This is the main method to derive when creating an operator. Context is the same dictionary used as when rendering jinja templates.

Refer to get_template_context for more context.

determine_kwargs(context)[source]
execute_callable()[source]

Calls the python callable with the given arguments.

Returns

the return value of the call.

Return type

Any

class airflow.operators.python.BranchPythonOperator(*, python_callable, op_args=None, op_kwargs=None, templates_dict=None, templates_exts=None, show_return_value_in_logs=True, **kwargs)[source]

Bases: PythonOperator, airflow.models.skipmixin.SkipMixin

Allows a workflow to “branch” or follow a path following the execution of this task.

It derives the PythonOperator and expects a Python function that returns a single task_id or list of task_ids to follow. The task_id(s) returned should point to a task directly downstream from {self}. All other “branches” or directly downstream tasks are marked with a state of skipped so that these paths can’t move forward. The skipped states are propagated downstream to allow for the DAG state to fill up and the DAG run’s state to be inferred.

execute(context)[source]

This is the main method to derive when creating an operator. Context is the same dictionary used as when rendering jinja templates.

Refer to get_template_context for more context.

class airflow.operators.python.ShortCircuitOperator(*, ignore_downstream_trigger_rules=True, **kwargs)[source]

Bases: PythonOperator, airflow.models.skipmixin.SkipMixin

Allows a pipeline to continue based on the result of a python_callable.

The ShortCircuitOperator is derived from the PythonOperator and evaluates the result of a python_callable. If the returned result is False or a falsy value, the pipeline will be short-circuited. Downstream tasks will be marked with a state of “skipped” based on the short-circuiting mode configured. If the returned result is True or a truthy value, downstream tasks proceed as normal and an XCom of the returned result is pushed.

The short-circuiting can be configured to either respect or ignore the trigger_rule set for downstream tasks. If ignore_downstream_trigger_rules is set to True, the default setting, all downstream tasks are skipped without considering the trigger_rule defined for tasks. However, if this parameter is set to False, the direct downstream tasks are skipped but the specified trigger_rule for other subsequent downstream tasks are respected. In this mode, the operator assumes the direct downstream tasks were purposely meant to be skipped but perhaps not other subsequent tasks.

See also

For more information on how to use this operator, take a look at the guide: ShortCircuitOperator

Parameters

ignore_downstream_trigger_rules (bool) – If set to True, all downstream tasks from this operator task will be skipped. This is the default behavior. If set to False, the direct, downstream task(s) will be skipped but the trigger_rule defined for a other downstream tasks will be respected.

execute(context)[source]

This is the main method to derive when creating an operator. Context is the same dictionary used as when rendering jinja templates.

Refer to get_template_context for more context.

class airflow.operators.python.PythonVirtualenvOperator(*, python_callable, requirements=None, python_version=None, use_dill=False, system_site_packages=True, pip_install_options=None, op_args=None, op_kwargs=None, string_args=None, templates_dict=None, templates_exts=None, expect_airflow=True, **kwargs)[source]

Bases: _BasePythonVirtualenvOperator

Allows one to run a function in a virtualenv that is created and destroyed automatically (with certain caveats).

The function must be defined using def, and not be part of a class. All imports must happen inside the function and no variables outside of the scope may be referenced. A global scope variable named virtualenv_string_args will be available (populated by string_args). In addition, one can pass stuff through op_args and op_kwargs, and one can use a return value. Note that if your virtualenv runs in a different Python major version than Airflow, you cannot use return values, op_args, op_kwargs, or use any macros that are being provided to Airflow through plugins. You can use string_args though.

See also

For more information on how to use this operator, take a look at the guide: PythonVirtualenvOperator

Parameters
  • python_callable (Callable) – A python function with no references to outside variables, defined with def, which will be run in a virtualenv

  • requirements (None | Iterable[str] | str) – Either a list of requirement strings, or a (templated) “requirements file” as specified by pip.

  • python_version (str | int | float | None) – The Python version to run the virtualenv with. Note that both 2 and 2.7 are acceptable forms.

  • use_dill (bool) – 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.

  • system_site_packages (bool) – Whether to include system_site_packages in your virtualenv. See virtualenv documentation for more information.

  • pip_install_options (list[str] | None) – a list of pip install options when installing requirements See ‘pip install -h’ for available options

  • op_args (Collection[Any] | None) – A list of positional arguments to pass to python_callable.

  • op_kwargs (Mapping[str, Any] | None) – A dict of keyword arguments to pass to python_callable.

  • string_args (Iterable[str] | None) – Strings that are present in the global var virtualenv_string_args, available to python_callable at runtime as a list[str]. Note that args are split by newline.

  • templates_dict (dict | None) – 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

  • templates_exts (list[str] | None) – a list of file extensions to resolve while processing templated fields, for examples ['.sql', '.hql']

  • expect_airflow (bool) – expect Airflow to be installed in the target environment. If true, the operator will raise warning if Airflow is not installed, and it will attempt to load Airflow macros when starting.

template_fields: Sequence[str][source]
template_ext: Sequence[str] = ('.txt',)[source]
execute_callable()[source]

Calls the python callable with the given arguments.

Returns

the return value of the call.

class airflow.operators.python.ExternalPythonOperator(*, python, python_callable, use_dill=False, op_args=None, op_kwargs=None, string_args=None, templates_dict=None, templates_exts=None, expect_airflow=True, expect_pendulum=False, **kwargs)[source]

Bases: _BasePythonVirtualenvOperator

Allows one to run a function in a virtualenv that is not re-created but used as is without the overhead of creating the virtualenv (with certain caveats).

The function must be defined using def, and not be part of a class. All imports must happen inside the function and no variables outside the scope may be referenced. A global scope variable named virtualenv_string_args will be available (populated by string_args). In addition, one can pass stuff through op_args and op_kwargs, and one can use a return value. Note that if your virtualenv runs in a different Python major version than Airflow, you cannot use return values, op_args, op_kwargs, or use any macros that are being provided to Airflow through plugins. You can use string_args though.

If Airflow is installed in the external environment in different version that the version used by the operator, the operator will fail.,

See also

For more information on how to use this operator, take a look at the guide: ExternalPythonOperator

Parameters
  • python (str) – 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).

  • python_callable (Callable) – A python function with no references to outside variables, defined with def, which will be run in a virtualenv

  • use_dill (bool) – Whether to use dill to serialize the args and result (pickle is default). This allow more complex types but if dill is not preinstalled in your venv, the task will fail with use_dill enabled.

  • op_args (Collection[Any] | None) – A list of positional arguments to pass to python_callable.

  • op_kwargs (Mapping[str, Any] | None) – A dict of keyword arguments to pass to python_callable.

  • string_args (Iterable[str] | None) – Strings that are present in the global var virtualenv_string_args, available to python_callable at runtime as a list[str]. Note that args are split by newline.

  • templates_dict (dict | None) – 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

  • templates_exts (list[str] | None) – a list of file extensions to resolve while processing templated fields, for examples ['.sql', '.hql']

  • expect_airflow (bool) – expect Airflow to be installed in the target environment. If true, the operator will raise warning if Airflow is not installed, and it will attempt to load Airflow macros when starting.

template_fields: Sequence[str][source]
execute_callable()[source]

Calls the python callable with the given arguments.

Returns

the return value of the call.

airflow.operators.python.get_current_context()[source]

Obtain the execution context for the currently executing operator without altering user method’s signature. This is the simplest method of retrieving the execution context dictionary.

Old style:

def my_task(**context):
    ti = context["ti"]

New style:

from airflow.operators.python import get_current_context


def my_task():
    context = get_current_context()
    ti = context["ti"]

Current context will only have value if this method was called after an operator was starting to execute.

Was this entry helpful?