Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
12 changes: 11 additions & 1 deletion task-sdk/src/airflow/sdk/definitions/decorators/task_group.py
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@
MappedArgument,
)
from airflow.sdk.definitions._internal.node import DAGNode
from airflow.sdk.definitions.mappedoperator import ensure_xcomarg_return_value
from airflow.sdk.definitions.taskgroup import MappedTaskGroup, TaskGroup
from airflow.sdk.definitions.xcom_arg import XComArg
from airflow.typing_compat import ParamSpec
Expand Down Expand Up @@ -135,6 +136,11 @@ def expand(self, **kwargs: OperatorExpandArgument) -> DAGNode:
self._validate_arg_names("expand", kwargs)
prevent_duplicates(self.partial_kwargs, kwargs, fail_reason="mapping already partial")
expand_input = DictOfListsExpandInput(kwargs)

# Similar to @task, @task_group should not be "mappable" over an XCom with a custom key. This will
# raise an exception, rather than having an ambiguous exception similar to the one found in #51109.
ensure_xcomarg_return_value(expand_input.value)

return self._create_task_group(
functools.partial(MappedTaskGroup, expand_input=expand_input),
**self.partial_kwargs,
Expand Down Expand Up @@ -162,8 +168,12 @@ def expand_kwargs(self, kwargs: OperatorExpandKwargsArgument) -> DAGNode:
# technically we can with AST but let's not), so we have to create stubs
# for every argument, including those with default values.
map_kwargs = (k for k in self.function_signature.parameters if k not in self.partial_kwargs)

expand_input = ListOfDictsExpandInput(kwargs)

# Similar to @task, @task_group should not be "mappable" over an XCom with a custom key. This will
# raise an exception, rather than having an ambiguous exception similar to the one found in #51109.
ensure_xcomarg_return_value(expand_input.value)

return self._create_task_group(
functools.partial(MappedTaskGroup, expand_input=expand_input),
**self.partial_kwargs,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@
import pytest

from airflow.providers.standard.operators.empty import EmptyOperator
from airflow.sdk import dag, task_group
from airflow.sdk import dag, task, task_group
from airflow.sdk.definitions._internal.expandinput import (
DictOfListsExpandInput,
ListOfDictsExpandInput,
Expand Down Expand Up @@ -158,6 +158,31 @@ def tg(a, b):
assert saved == {"a": 1, "b": MappedArgument(input=tg._expand_input, key="b")}


def test_expand_invalid_xcomarg_return_value():
saved = {}

@dag(schedule=None, start_date=pendulum.datetime(2022, 1, 1))
def pipeline():
@task
def t():
return {"values": ["value_1", "value_2"]}

@task_group()
def tg(a, b):
saved["a"] = a
saved["b"] = b

tg.partial(a=1).expand(b=t()["values"])

with pytest.raises(ValueError) as ctx:
pipeline()

assert (
str(ctx.value)
== "cannot map over XCom with custom key 'values' from <Task(_PythonDecoratedOperator): t>"
)


def test_expand_kwargs_no_wildcard():
@dag(schedule=None, start_date=pendulum.datetime(2022, 1, 1))
def pipeline():
Expand Down Expand Up @@ -194,6 +219,31 @@ def tg(a, b):
assert saved == {"a": 1, "b": MappedArgument(input=tg._expand_input, key="b")}


def test_expand_kwargs_invalid_xcomarg_return_value():
saved = {}

@dag(schedule=None, start_date=pendulum.datetime(2022, 1, 1))
def pipeline():
@task
def t():
return {"values": [{"b": 2}, {"b": 3}]}

@task_group()
def tg(a, b):
saved["a"] = a
saved["b"] = b

tg.partial(a=1).expand_kwargs(t()["values"])

with pytest.raises(ValueError) as ctx:
pipeline()

assert (
str(ctx.value)
== "cannot map over XCom with custom key 'values' from <Task(_PythonDecoratedOperator): t>"
)


def test_override_dag_default_args():
@dag(
dag_id="test_dag",
Expand Down