Skip to content
Closed
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
Original file line number Diff line number Diff line change
Expand Up @@ -36,3 +36,9 @@ class DagRunStateResponse(BaseModel):
"""Schema for DAG Run State response."""

state: DagRunState


class DagRunCountResponse(BaseModel):
"""Schema for DAG Count response."""

count: int
Original file line number Diff line number Diff line change
Expand Up @@ -18,14 +18,20 @@
from __future__ import annotations

import logging
from datetime import datetime
from typing import Annotated

from fastapi import HTTPException, status
from sqlalchemy import select
from fastapi import HTTPException, Query, status
from sqlalchemy import func, select

from airflow.api.common.trigger_dag import trigger_dag
from airflow.api_fastapi.common.db.common import SessionDep
from airflow.api_fastapi.common.router import AirflowRouter
from airflow.api_fastapi.execution_api.datamodels.dagrun import DagRunStateResponse, TriggerDAGRunPayload
from airflow.api_fastapi.execution_api.datamodels.dagrun import (
DagRunCountResponse,
DagRunStateResponse,
TriggerDAGRunPayload,
)
from airflow.exceptions import DagRunAlreadyExists
from airflow.models.dag import DagModel
from airflow.models.dagbag import DagBag
Expand Down Expand Up @@ -150,3 +156,39 @@ def get_dagrun_state(
)

return DagRunStateResponse(state=dag_run.state)


@router.get(
"/{dag_id}/count",
responses={
status.HTTP_404_NOT_FOUND: {"description": "DAG not found for the given dag_id"},
},
)
def get_dag_count(
dag_id: str,
session: SessionDep,
run_ids: Annotated[list[str] | None, Query()] = None,
states: Annotated[list[str] | None, Query()] = None,
logical_dates: Annotated[list[datetime] | None, Query()] = None,
) -> DagRunCountResponse:
"""Get the count of DAGs by run_ids and states."""
dm = session.scalar(select(DagModel).where(DagModel.is_active, DagModel.dag_id == dag_id).limit(1))
if not dm:
raise HTTPException(
status.HTTP_404_NOT_FOUND,
detail={"reason": "not_found", "message": f"DAG with dag_id: '{dag_id}' not found"},
)

query = select(func.count()).select_from(DagRun).where(DagRun.dag_id == dag_id)

if run_ids:
query = query.where(DagRun.run_id.in_(run_ids))

if states:
query = query.where(DagRun.state.in_(states))

if logical_dates:
query = query.where(DagRun.logical_date.in_(logical_dates))

result = session.scalar(query)
return DagRunCountResponse(count=result or 0)
24 changes: 24 additions & 0 deletions airflow-core/src/airflow/exceptions.py
Original file line number Diff line number Diff line change
Expand Up @@ -434,6 +434,7 @@ def __init__(
allowed_states: list[str | DagRunState],
failed_states: list[str | DagRunState],
poke_interval: int,
deferrable: bool,
):
super().__init__()
self.trigger_dag_id = trigger_dag_id
Expand All @@ -446,6 +447,29 @@ def __init__(
self.allowed_states = allowed_states
self.failed_states = failed_states
self.poke_interval = poke_interval
self.deferrable = deferrable


class DagRunTriggerExecuteCompleteException(AirflowException):
"""
Signal by an operator execute complete method.

Special exception raised to signal that the operator ``TriggerDagRunOperator`` to execute_complete method.
"""

def __init__(
self,
*,
trigger_dag_id: str,
run_ids: str,
allowed_states: list[str | DagRunState],
failed_states: list[str | DagRunState],
):
super().__init__()
self.trigger_dag_id = trigger_dag_id
self.run_ids = run_ids
self.allowed_states = allowed_states
self.failed_states = failed_states


class TaskDeferred(BaseException):
Expand Down
24 changes: 22 additions & 2 deletions airflow-core/src/airflow/jobs/triggerer_job_runner.py
Original file line number Diff line number Diff line change
Expand Up @@ -44,8 +44,10 @@
from airflow.models.trigger import Trigger
from airflow.sdk.execution_time.comms import (
ConnectionResult,
DagRunCountResult,
ErrorResponse,
GetConnection,
GetDagRunCount,
GetVariable,
GetXCom,
VariableResult,
Expand Down Expand Up @@ -212,6 +214,7 @@ class TriggerStateChanges(BaseModel):
ConnectionResult,
VariableResult,
XComResult,
DagRunCountResult,
ErrorResponse,
],
Field(discriminator="type"),
Expand All @@ -223,7 +226,7 @@ class TriggerStateChanges(BaseModel):


ToTriggerSupervisor = Annotated[
Union[messages.TriggerStateChanges, GetConnection, GetVariable, GetXCom],
Union[messages.TriggerStateChanges, GetConnection, GetVariable, GetXCom, GetDagRunCount],
Field(discriminator="type"),
]
"""
Expand Down Expand Up @@ -333,7 +336,12 @@ def client(self) -> Client:
return client

def _handle_request(self, msg: ToTriggerSupervisor, log: FilteringBoundLogger) -> None: # type: ignore[override]
from airflow.sdk.api.datamodels._generated import ConnectionResponse, VariableResponse, XComResponse
from airflow.sdk.api.datamodels._generated import (
ConnectionResponse,
DagRunCountResponse,
VariableResponse,
XComResponse,
)

resp = None

Expand Down Expand Up @@ -371,6 +379,18 @@ def _handle_request(self, msg: ToTriggerSupervisor, log: FilteringBoundLogger) -
resp = xcom_result.model_dump_json(exclude_unset=True).encode()
else:
resp = xcom.model_dump_json().encode()

elif isinstance(msg, GetDagRunCount):
dr_resp = self.client.dag_runs.get_dag_run_count(
msg.dag_id, msg.run_ids, msg.states, msg.logical_dates
)

if isinstance(dr_resp, DagRunCountResponse):
dag_run_state_count_result = DagRunCountResult.from_api_response(dr_resp)
resp = dag_run_state_count_result.model_dump_json(exclude_unset=True).encode()
else:
resp = dr_resp.model_dump_json().encode()

else:
raise ValueError(f"Unknown message type {type(msg)}")

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -218,3 +218,105 @@ def test_dag_run_not_found(self, client):
response = client.post(f"/execution/dag-runs/{dag_id}/{run_id}/clear")

assert response.status_code == 404


class TestDagRunCount:
def setup_method(self):
clear_db_runs()

def teardown_method(self):
clear_db_runs()

def test_dag_run_count(self, client, session, dag_maker):
dag_id = "test_dag_run_count_by_run_ids_and_states"
run_id1 = "test_run_id1"
run_id2 = "test_run_id2"

with dag_maker(dag_id=dag_id, session=session, serialized=True):
EmptyOperator(task_id="test_task")

dag_maker.create_dagrun(
run_id=run_id1, state=DagRunState.SUCCESS, logical_date=timezone.datetime(2025, 2, 20)
)
dag_maker.create_dagrun(
run_id=run_id2, state=DagRunState.SUCCESS, logical_date=timezone.datetime(2025, 3, 20)
)

session.commit()

response = client.get(
f"/execution/dag-runs/{dag_id}/count",
params={"run_ids": [run_id1, run_id2], "states": [DagRunState.SUCCESS]},
)

assert response.status_code == 200
assert response.json() == {"count": 2}

def test_dag_run_count_by_run_ids_and_success_failure_states(self, client, session, dag_maker):
dag_id = "test_dag_run_count_by_run_ids_and_states"
run_id1 = "test_run_id3"
run_id2 = "test_run_id4"

with dag_maker(dag_id=dag_id, session=session, serialized=True):
EmptyOperator(task_id="test_task")

dag_maker.create_dagrun(
run_id=run_id1, state=DagRunState.SUCCESS, logical_date=timezone.datetime(2025, 4, 20)
)
dag_maker.create_dagrun(
run_id=run_id2, state=DagRunState.FAILED, logical_date=timezone.datetime(2025, 5, 20)
)

session.commit()

response = client.get(
f"/execution/dag-runs/{dag_id}/count",
params={"run_ids": [run_id1, run_id2], "states": [DagRunState.SUCCESS, DagRunState.FAILED]},
)

assert response.status_code == 200
assert response.json() == {"count": 2}

def test_dag_run_count_by_logical_dates_and_success_failure_states(self, client, session, dag_maker):
dag_id = "dag_run_count_by_logical_dates_and_success_failure_states"
run_one_logical_date = timezone.datetime(2025, 4, 20)
run_two_logical_date = timezone.datetime(2025, 5, 20)

with dag_maker(dag_id=dag_id, session=session, serialized=True):
EmptyOperator(task_id="test_task")

dag_maker.create_dagrun(
run_id="test_run_id5", state=DagRunState.SUCCESS, logical_date=timezone.datetime(2025, 4, 20)
)
dag_maker.create_dagrun(
run_id="test_run_id6", state=DagRunState.FAILED, logical_date=timezone.datetime(2025, 5, 20)
)

session.commit()

response = client.get(
f"/execution/dag-runs/{dag_id}/count",
params={
"logical_dates": [run_one_logical_date, run_two_logical_date],
"states": [DagRunState.SUCCESS, DagRunState.FAILED],
},
)

assert response.status_code == 200
assert response.json() == {"count": 2}

def test_dag_run_count_by_run_ids_and_states_dag_not_found(self, client):
dag_id = "dag_not_found"

response = client.get(
f"/execution/dag-runs/{dag_id}/count",
params={"run_ids": ["test_run_id1"], "states": [DagRunState.SUCCESS]},
)

assert response.status_code == 404
assert response.json() == {
"detail": {
"message": "DAG with dag_id: 'dag_not_found' not found",
"reason": "not_found",
}
}
78 changes: 78 additions & 0 deletions airflow-core/tests/unit/jobs/test_triggerer_job.py
Original file line number Diff line number Diff line change
Expand Up @@ -734,3 +734,81 @@ async def test_trigger_can_access_variables_connections_and_xcoms(session, dag_m
"xcom": '"some_xcom_value"',
}
}


class CustomTriggerDagRun(BaseTrigger):
def __init__(self, trigger_dag_id, run_ids, states, logical_dates):
self.trigger_dag_id = trigger_dag_id
self.run_ids = run_ids
self.states = states
self.logical_dates = logical_dates

def serialize(self) -> tuple[str, dict[str, Any]]:
return (
f"{type(self).__module__}.{type(self).__qualname__}",
{
"trigger_dag_id": self.trigger_dag_id,
"run_ids": self.run_ids,
"states": self.states,
"logical_dates": self.logical_dates,
},
)

async def run(self, **args) -> AsyncIterator[TriggerEvent]:
from airflow.sdk.execution_time.context import get_dag_run_count

print(self.trigger_dag_id, self.run_ids, self.states)
dag_run_states_count = await sync_to_async(get_dag_run_count)(
dag_id=self.trigger_dag_id,
run_ids=self.run_ids,
states=self.states,
logical_dates=self.logical_dates,
)
yield TriggerEvent({"count": dag_run_states_count.count})


@pytest.mark.xfail(
reason="We know that test is flaky and have no time to fix it before 3.0. "
"We should fix it later. TODO: AIP-72"
)
@pytest.mark.asyncio
@pytest.mark.flaky(reruns=2, reruns_delay=10)
@pytest.mark.execution_timeout(30)
async def test_trigger_can_fetch_trigger_dag_run_count_in_deferrable(session, dag_maker):
"""Checks that the trigger will successfully fetch the count of trigger DAG runs."""
# Create the test DAG and task
with dag_maker(dag_id="trigger_can_fetch_trigger_dag_run_count_in_deferrable", session=session):
EmptyOperator(task_id="dummy1")
dr = dag_maker.create_dagrun()
task_instance = dr.task_instances[0]
task_instance.state = TaskInstanceState.DEFERRED

# Use the same dag run with states deferred to fetch the count
trigger = CustomTriggerDagRun(
trigger_dag_id=dr.dag_id, run_ids=[dr.run_id], states=[dr.state], logical_dates=[dr.logical_date]
)
trigger_orm = Trigger(
classpath=trigger.serialize()[0],
kwargs={
"trigger_dag_id": dr.dag_id,
"run_ids": [dr.run_id],
"states": [dr.state],
"logical_dates": [dr.logical_date],
},
)
trigger_orm.id = 1
session.add(trigger_orm)
session.commit()
task_instance.trigger_id = trigger_orm.id

job = Job()
session.add(job)
session.commit()

supervisor = DummyTriggerRunnerSupervisor.start(job=job, capacity=1, logger=None)
supervisor.run()

task_instance.refresh_from_db()
assert task_instance.state == TaskInstanceState.SCHEDULED
assert task_instance.next_method != "__fail__"
assert task_instance.next_kwargs == {"event": {"count": 1}}
Loading
Loading