Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Fix clearing child dag mapped tasks from parent dag #27501

Merged
merged 4 commits into from
Nov 17, 2022
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
2 changes: 1 addition & 1 deletion airflow/models/dag.py
Original file line number Diff line number Diff line change
Expand Up @@ -1773,7 +1773,7 @@ def _get_task_instances(
if result or as_pk_tuple:
# Only execute the `ti` query if we have also collected some other results (i.e. subdags etc.)
if as_pk_tuple:
result.update(TaskInstanceKey(*cols) for cols in tis.all())
result.update(TaskInstanceKey(**cols._mapping) for cols in tis.all())
else:
result.update(ti.key for ti in tis)

Expand Down
93 changes: 93 additions & 0 deletions tests/sensors/test_external_task_sensor.py
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@
import pytest

from airflow import exceptions, settings
from airflow.decorators import task as task_deco
from airflow.exceptions import AirflowException, AirflowSensorTimeout
from airflow.models import DagBag, DagRun, TaskInstance
from airflow.models.dag import DAG
Expand Down Expand Up @@ -1172,6 +1173,98 @@ def test_clear_overlapping_external_task_marker(dag_bag_head_tail, session):
)


@pytest.fixture
def dag_bag_head_tail_mapped_tasks():
"""
Create a DagBag containing one DAG, with task "head" depending on task "tail" of the
previous execution_date.

20200501 20200502 20200510
+------+ +------+ +------+
| head | -->head | --> -->head |
| | | / | | | / / | | |
| v | / | v | / / | v |
| body | / | body | / ... / | body |
| | |/ | | |/ / | | |
| v / | v / / | v |
| tail/| | tail/| / | tail |
+------+ +------+ +------+
"""
dag_bag = DagBag(dag_folder=DEV_NULL, include_examples=False)

with DAG("head_tail", start_date=DEFAULT_DATE, schedule="@daily") as dag:

@task_deco
def dummy_task(x: int):
return x

head = ExternalTaskSensor(
task_id="head",
external_dag_id=dag.dag_id,
external_task_id="tail",
execution_delta=timedelta(days=1),
mode="reschedule",
)

body = dummy_task.expand(x=[i for i in range(5)])
tail = ExternalTaskMarker(
task_id="tail",
external_dag_id=dag.dag_id,
external_task_id=head.task_id,
execution_date="{{ macros.ds_add(ds, 1) }}",
)
head >> body >> tail

dag_bag.bag_dag(dag=dag, root_dag=dag)

return dag_bag


@provide_session
def test_clear_overlapping_external_task_marker_mapped_tasks(dag_bag_head_tail_mapped_tasks, session):
dag: DAG = dag_bag_head_tail_mapped_tasks.get_dag("head_tail")

# "Run" 10 times.
for delta in range(0, 10):
execution_date = DEFAULT_DATE + timedelta(days=delta)
dagrun = DagRun(
dag_id=dag.dag_id,
state=DagRunState.SUCCESS,
execution_date=execution_date,
run_type=DagRunType.MANUAL,
run_id=f"test_{delta}",
)
session.add(dagrun)
for task in dag.tasks:
if task.task_id == "dummy_task":
for map_index in range(5):
ti = TaskInstance(task=task, run_id=dagrun.run_id, map_index=map_index)
ti.state = TaskInstanceState.SUCCESS
dagrun.task_instances.append(ti)
else:
ti = TaskInstance(task=task, run_id=dagrun.run_id)
ti.state = TaskInstanceState.SUCCESS
dagrun.task_instances.append(ti)
session.flush()

dag = dag.partial_subset(
task_ids_or_regex=["head"],
include_downstream=True,
include_upstream=False,
)
task_ids = [tid for tid in dag.task_dict]
assert (
dag.clear(
start_date=DEFAULT_DATE,
end_date=DEFAULT_DATE,
dag_bag=dag_bag_head_tail_mapped_tasks,
session=session,
task_ids=task_ids,
)
== 70
)


class TestExternalTaskSensorLink:
def test_deprecation_warning(self):
with pytest.warns(DeprecationWarning) as warnings:
Expand Down