amoghrajesh commented on code in PR #53655:
URL: https://github.com/apache/airflow/pull/53655#discussion_r2225143908
##########
airflow-core/src/airflow/api_fastapi/execution_api/routes/dag_runs.py:
##########
@@ -172,16 +177,45 @@ def get_dr_count(
states: Annotated[list[str] | None, Query()] = None,
) -> int:
"""Get the count of DAG runs matching the given criteria."""
- query = select(func.count()).select_from(DagRun).where(DagRun.dag_id ==
dag_id)
+ query =
select(func.count()).select_from(DagRunModel).where(DagRunModel.dag_id ==
dag_id)
if logical_dates:
- query = query.where(DagRun.logical_date.in_(logical_dates))
+ query = query.where(DagRunModel.logical_date.in_(logical_dates))
if run_ids:
- query = query.where(DagRun.run_id.in_(run_ids))
+ query = query.where(DagRunModel.run_id.in_(run_ids))
if states:
- query = query.where(DagRun.state.in_(states))
+ query = query.where(DagRunModel.state.in_(states))
count = session.scalar(query)
return count or 0
+
+
[email protected]("/{dag_id}/previous", status_code=status.HTTP_200_OK)
+def get_previous_dagrun(
+ dag_id: str,
+ logical_date: UtcDateTime,
+ session: SessionDep,
+ state: Annotated[str | None, Query()] = None,
+) -> DagRun | None:
+ """Get the previous DAG run before the given logical date, optionally
filtered by state."""
+ query = (
+ select(DagRunModel)
+ .where(
+ DagRunModel.dag_id == dag_id,
+ DagRunModel.logical_date < logical_date,
+ )
+ .order_by(DagRunModel.logical_date.desc())
+ .limit(1)
+ )
+
+ if state:
Review Comment:
Validate if a valid state is sent for filtering? (check in `DagRunState`)
Nothing stops me from sending "abcd" as state.
##########
airflow-core/src/airflow/api_fastapi/execution_api/versions/v2025_08_10.py:
##########
@@ -28,3 +28,20 @@ class AddDagVersionIdField(VersionChange):
description = __doc__
instructions_to_migrate_to_previous_version =
(schema(TaskInstance).field("dag_version_id").didnt_exist,)
+
+
+class AddDagRunStateFieldAndPreviousEndpoint(VersionChange):
+ """Add the `state` field to DagRun model and `/dag-runs/{dag_id}/previous`
endpoint."""
+
+ description = __doc__
+
+ instructions_to_migrate_to_previous_version = (
+ schema(DagRun).field("state").didnt_exist,
+ endpoint("/dag-runs/{dag_id}/previous", ["GET"]).didnt_exist,
+ )
+
+ @convert_response_to_previous_version_for(TIRunContext) # type:
ignore[arg-type]
+ def remove_state_from_dag_run(response: ResponseInfo = None) -> None: #
type: ignore
+ """Remove the `state` field from the dag_run object when converting to
the previous version."""
+ if "dag_run" in response.body and isinstance(response.body["dag_run"],
dict):
+ response.body["dag_run"].pop("state", None)
Review Comment:
Looks good.
##########
airflow-core/src/airflow/api_fastapi/execution_api/versions/v2025_08_10.py:
##########
@@ -28,3 +28,20 @@ class AddDagVersionIdField(VersionChange):
description = __doc__
instructions_to_migrate_to_previous_version =
(schema(TaskInstance).field("dag_version_id").didnt_exist,)
+
+
+class AddDagRunStateFieldAndPreviousEndpoint(VersionChange):
+ """Add the `state` field to DagRun model and `/dag-runs/{dag_id}/previous`
endpoint."""
+
+ description = __doc__
+
+ instructions_to_migrate_to_previous_version = (
+ schema(DagRun).field("state").didnt_exist,
+ endpoint("/dag-runs/{dag_id}/previous", ["GET"]).didnt_exist,
+ )
+
+ @convert_response_to_previous_version_for(TIRunContext) # type:
ignore[arg-type]
+ def remove_state_from_dag_run(response: ResponseInfo = None) -> None: #
type: ignore
Review Comment:
Why are the type ignores needed?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]