Skip to content

Commit

Permalink
Remove auto unroll for dict values from taskflow
Browse files Browse the repository at this point in the history
dict return values from functions were automatically unrolled
in separate values. This meant that the default of `multiple_outputs`
was changing based on the return type of the function.

This is confusing as it requires the user to be aware of this and
rather un-pythonic (explicit over implicit) and makes conversion from
plain python functions to Airflow tasks harder as unit tests would suddenly
fail.

Closes: apache#27819
  • Loading branch information
bolkedebruin committed Nov 22, 2022
1 parent 093345c commit eac885b
Show file tree
Hide file tree
Showing 3 changed files with 9 additions and 14 deletions.
11 changes: 1 addition & 10 deletions airflow/decorators/base.py
Original file line number Diff line number Diff line change
Expand Up @@ -286,22 +286,13 @@ class _TaskDecorator(ExpandableFactory, Generic[FParams, FReturn, OperatorSubcla

function: Callable[FParams, FReturn] = attr.ib(validator=attr.validators.is_callable())
operator_class: type[OperatorSubclass]
multiple_outputs: bool = attr.ib()
multiple_outputs: bool = attr.ib(default=False)
kwargs: dict[str, Any] = attr.ib(factory=dict)

decorator_name: str = attr.ib(repr=False, default="task")

_airflow_is_task_decorator: ClassVar[bool] = True

@multiple_outputs.default
def _infer_multiple_outputs(self):
try:
return_type = typing_extensions.get_type_hints(self.function).get("return", Any)
except TypeError: # Can't evaluate return type.
return False
ttype = getattr(return_type, "__origin__", return_type)
return ttype == dict or ttype == Dict

def __attrs_post_init__(self):
if "self" in self.function_signature.parameters:
raise TypeError(f"@{self.decorator_name} does not support methods")
Expand Down
10 changes: 6 additions & 4 deletions docs/apache-airflow/tutorial/taskflow.rst
Original file line number Diff line number Diff line change
Expand Up @@ -377,19 +377,21 @@ section "Having sensors return XOM values" of :doc:`apache-airflow-providers:how

Multiple outputs inference
--------------------------
Tasks can also infer multiple outputs by using dict Python typing.
In case a task has multiple return values, Airflow can unroll these in separate values if using
Python dict typing. The values then become available as separate arguments to downstream tasks.

.. code-block:: python
@task
@task(multiple_outputs=True)
def identity_dict(x: int, y: int) -> dict[str, int]:
return {"x": x, "y": y}
By using the typing ``Dict`` for the function return type, the ``multiple_outputs`` parameter
is automatically set to true.

Note, If you manually set the ``multiple_outputs`` parameter the inference is disabled and
the parameter value is used.
Note, before Airflow 2.5, Airflow would automatically unroll ``dict`` return values and you would need to set
``multiple_outputs=False` explicitly. This was deemed confusing and now always defaults to ``False``.


Adding dependencies between decorated and traditional tasks
-----------------------------------------------------------
Expand Down
2 changes: 2 additions & 0 deletions newsfragments/27819.significant.rst
Original file line number Diff line number Diff line change
@@ -0,0 +1,2 @@
Airflow no longer automatically unrolls dict values into separate arguments requiring an explicit
``multiple_outputs=True`` if unrolling is required.

0 comments on commit eac885b

Please sign in to comment.