Skip to content
Merged
Show file tree
Hide file tree
Changes from 5 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
@@ -0,0 +1,32 @@
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.

from __future__ import annotations

from airflow.api_fastapi.core_api.base import StrictBaseModel


class AssetStateResponse(StrictBaseModel):
"""Asset state value returned to a worker."""

value: str


class AssetStatePutBody(StrictBaseModel):
"""Request body for setting an asset state value."""

value: str
Original file line number Diff line number Diff line change
@@ -0,0 +1,32 @@
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.

from __future__ import annotations

from airflow.api_fastapi.core_api.base import StrictBaseModel


class TaskStateResponse(StrictBaseModel):
"""Task state value returned to a worker."""

value: str


class TaskStatePutBody(StrictBaseModel):
"""Request body for setting a task state value."""

value: str
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@

from airflow.api_fastapi.execution_api.routes import (
asset_events,
asset_state,
assets,
connections,
dag_runs,
Expand All @@ -29,6 +30,7 @@
hitl,
task_instances,
task_reschedules,
task_state,
variables,
xcoms,
)
Expand All @@ -52,5 +54,7 @@
authenticated_router.include_router(variables.router, prefix="/variables", tags=["Variables"])
authenticated_router.include_router(xcoms.router, prefix="/xcoms", tags=["XComs"])
authenticated_router.include_router(hitl.router, prefix="/hitlDetails", tags=["Human in the Loop"])
authenticated_router.include_router(task_state.router, prefix="/state/ti", tags=["Task State"])
authenticated_router.include_router(asset_state.router, prefix="/state/asset", tags=["Asset State"])
Comment thread
kaxil marked this conversation as resolved.

execution_api_router.include_router(authenticated_router)
Original file line number Diff line number Diff line change
@@ -0,0 +1,121 @@
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
"""
Execution API routes for asset state.

Asset state is keyed by asset *name* (not integer id) in the URL — asset names
are unique, and callers (task SDK accessors) have the name from their Asset
object without needing a DB lookup. The route resolves name → asset_id
internally for the state backend scope.

Per-task asset registration checks are intentionally not implemented here
(deferred to AIP-93 — see TODO comment below).
"""

from __future__ import annotations

from typing import Annotated

from cadwyn import VersionedAPIRouter
from fastapi import HTTPException, Path, status
from sqlalchemy import select

from airflow._shared.state import AssetScope
from airflow.api_fastapi.common.db.common import SessionDep
from airflow.api_fastapi.execution_api.datamodels.asset_state import (
AssetStatePutBody,
AssetStateResponse,
)
from airflow.models.asset import AssetModel
from airflow.state import get_state_backend

# TODO(AIP-103): enforce that the requesting task is registered with the asset
Comment thread
amoghrajesh marked this conversation as resolved.
# (via task_inlet_asset_reference or task_outlet_asset_reference) before
# allowing reads/writes. Currently any task with a valid execution token can
# access any asset's state — the same gap exists in /assets and /asset-events.
# Proper fix is a unified asset-registration check across all asset routes,
# not just here.
router = VersionedAPIRouter(
responses={
status.HTTP_401_UNAUTHORIZED: {"description": "Unauthorized"},
status.HTTP_404_NOT_FOUND: {"description": "Not found"},
},
)


def _resolve_asset_id(name: str, session: SessionDep) -> int:
"""Resolve asset name → integer asset_id, 404 if not found."""
asset_id = session.scalar(select(AssetModel.id).where(AssetModel.name == name, AssetModel.active.has()))
if asset_id is None:
raise HTTPException(
status_code=status.HTTP_404_NOT_FOUND,
detail={"reason": "not_found", "message": f"Asset {name!r} not found"},
)
return asset_id


@router.get("/{name}/{key}")
Comment thread
amoghrajesh marked this conversation as resolved.
Outdated
def get_asset_state(
name: Annotated[str, Path(min_length=1)],
key: Annotated[str, Path(min_length=1)],
session: SessionDep,
) -> AssetStateResponse:
"""Get an asset state value."""
asset_id = _resolve_asset_id(name, session)
value = get_state_backend().get(AssetScope(asset_id=asset_id), key, session=session)
if value is None:
raise HTTPException(
status_code=status.HTTP_404_NOT_FOUND,
detail={
"reason": "not_found",
"message": f"Asset state key {key!r} not found",
},
)
return AssetStateResponse(value=value)


@router.put("/{name}/{key}", status_code=status.HTTP_204_NO_CONTENT)
def set_asset_state(
name: Annotated[str, Path(min_length=1)],
key: Annotated[str, Path(min_length=1)],
body: AssetStatePutBody,
session: SessionDep,
) -> None:
"""Set an asset state value."""
asset_id = _resolve_asset_id(name, session)
get_state_backend().set(AssetScope(asset_id=asset_id), key, body.value, session=session)


@router.delete("/{name}/{key}", status_code=status.HTTP_204_NO_CONTENT)
def delete_asset_state(
name: Annotated[str, Path(min_length=1)],
key: Annotated[str, Path(min_length=1)],
session: SessionDep,
) -> None:
"""Delete a single asset state key."""
asset_id = _resolve_asset_id(name, session)
get_state_backend().delete(AssetScope(asset_id=asset_id), key, session=session)


@router.delete("/{name}", status_code=status.HTTP_204_NO_CONTENT)
def clear_asset_state(
name: Annotated[str, Path(min_length=1)],
session: SessionDep,
) -> None:
"""Delete all state keys for an asset."""
asset_id = _resolve_asset_id(name, session)
get_state_backend().clear(AssetScope(asset_id=asset_id), session=session)
Original file line number Diff line number Diff line change
@@ -0,0 +1,136 @@
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
from __future__ import annotations

from typing import Annotated
from uuid import UUID

from cadwyn import VersionedAPIRouter
from fastapi import HTTPException, Path, Query, Security, status
from sqlalchemy.orm import Session

from airflow._shared.state import TaskScope
from airflow.api_fastapi.common.db.common import SessionDep
from airflow.api_fastapi.execution_api.datamodels.task_state import (
TaskStatePutBody,
TaskStateResponse,
)
from airflow.api_fastapi.execution_api.security import require_auth
from airflow.models.taskinstance import TaskInstance as TI
from airflow.state import get_state_backend

router = VersionedAPIRouter(
Comment thread
amoghrajesh marked this conversation as resolved.
responses={
status.HTTP_401_UNAUTHORIZED: {"description": "Unauthorized"},
status.HTTP_403_FORBIDDEN: {"description": "Access denied"},
status.HTTP_404_NOT_FOUND: {"description": "Not found"},
},
dependencies=[Security(require_auth, scopes=["ti:self"])],
Comment thread
amoghrajesh marked this conversation as resolved.
Comment thread
amoghrajesh marked this conversation as resolved.
)


def _get_task_scope_for_ti(task_instance_id: UUID, session: Session) -> TaskScope:
ti = session.get(TI, task_instance_id)
if ti is None:
raise HTTPException(
status_code=status.HTTP_404_NOT_FOUND,
detail={
"reason": "not_found",
"message": f"Task instance {task_instance_id} not found",
},
)
return TaskScope(dag_id=ti.dag_id, run_id=ti.run_id, task_id=ti.task_id, map_index=ti.map_index)


@router.get("/{task_instance_id}/{key}")
def get_task_state(
task_instance_id: UUID,
key: Annotated[str, Path(min_length=1)],
session: SessionDep,
) -> TaskStateResponse:
"""Get value for a task state."""
scope = _get_task_scope_for_ti(task_instance_id, session)
value = get_state_backend().get(scope, key, session=session)
if value is None:
raise HTTPException(
status_code=status.HTTP_404_NOT_FOUND,
detail={
"reason": "not_found",
"message": f"Task state key {key!r} not found",
},
)
return TaskStateResponse(value=value)


@router.put("/{task_instance_id}/{key}", status_code=status.HTTP_204_NO_CONTENT)
def set_task_state(
task_instance_id: UUID,
key: Annotated[str, Path(min_length=1)],
body: TaskStatePutBody,
session: SessionDep,
) -> None:
"""Set a task state key, creating or updating the row."""
scope = _get_task_scope_for_ti(task_instance_id, session)
get_state_backend().set(scope, key, body.value, session=session)


@router.delete("/{task_instance_id}/{key}", status_code=status.HTTP_204_NO_CONTENT)
def delete_task_state(
task_instance_id: UUID,
key: Annotated[str, Path(min_length=1)],
session: SessionDep,
) -> None:
"""Delete a single task state key."""
scope = _get_task_scope_for_ti(task_instance_id, session)
get_state_backend().delete(scope, key, session=session)


@router.delete("/{task_instance_id}", status_code=status.HTTP_204_NO_CONTENT)
def clear_task_state(
task_instance_id: UUID,
session: SessionDep,
all_map_indices: Annotated[bool, Query()] = False,
Comment thread
amoghrajesh marked this conversation as resolved.
) -> None:
"""
Delete all task state keys for this task instance.

By default, only keys for the requesting TI's exact ``map_index`` are
cleared — same isolation as DELETE endpoint above.

Pass ``?all_map_indices=true`` to wipe state for every mapped sibling of
the task in the same DAG run. This is intentionally fleet-wide: the
``ti:self`` JWT authentication scope authenticates that the caller is
a legitimate member of the mapped task group, and grants it authority
to reset shared task state on behalf of the whole group.
The SDK only forwards this flag when the user calls ``task_state.clear(all_map_indices=True)``
explicitly, so the expanded scope is always an explicit opt-in by the task author.

For non-mapped tasks (``map_index=-1``), there is only ever one index, so
``?all_map_indices=true`` is functionally identical to the default and is
accepted without error.
"""
ti = session.get(TI, task_instance_id)
if ti is None:
raise HTTPException(
status_code=status.HTTP_404_NOT_FOUND,
detail={
"reason": "not_found",
"message": f"Task instance {task_instance_id} not found",
},
)
scope = TaskScope(dag_id=ti.dag_id, run_id=ti.run_id, task_id=ti.task_id, map_index=ti.map_index)
Comment thread
amoghrajesh marked this conversation as resolved.
Outdated
get_state_backend().clear(scope, all_map_indices=all_map_indices, session=session)
Original file line number Diff line number Diff line change
Expand Up @@ -40,13 +40,14 @@
MovePreviousRunEndpoint,
RemoveUpstreamMapIndexesField,
)
from airflow.api_fastapi.execution_api.versions.v2026_04_17 import AddTeamNameField
from airflow.api_fastapi.execution_api.versions.v2026_04_17 import AddStateEndpoints, AddTeamNameField

bundle = VersionBundle(
HeadVersion(),
Version(
"2026-04-17",
AddTeamNameField,
AddStateEndpoints,
),
Version(
"2026-04-06",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

from __future__ import annotations

from cadwyn import ResponseInfo, VersionChange, convert_response_to_previous_version_for, schema
from cadwyn import ResponseInfo, VersionChange, convert_response_to_previous_version_for, endpoint, schema

from airflow.api_fastapi.execution_api.datamodels.taskinstance import DagRun, TIRunContext

Expand All @@ -34,3 +34,20 @@ def remove_team_name_field(response: ResponseInfo) -> None: # type: ignore[misc
"""Remove the ``team_name`` field from dag_run for older API versions."""
if "dag_run" in response.body and isinstance(response.body["dag_run"], dict):
response.body["dag_run"].pop("team_name", None)


class AddStateEndpoints(VersionChange):
"""Add task state and asset state CRUD endpoints."""

description = __doc__

instructions_to_migrate_to_previous_version = (
endpoint("/state/ti/{task_instance_id}/{key}", ["GET"]).didnt_exist,
endpoint("/state/ti/{task_instance_id}/{key}", ["PUT"]).didnt_exist,
endpoint("/state/ti/{task_instance_id}/{key}", ["DELETE"]).didnt_exist,
endpoint("/state/ti/{task_instance_id}", ["DELETE"]).didnt_exist,
endpoint("/state/asset/{name}/{key}", ["GET"]).didnt_exist,
endpoint("/state/asset/{name}/{key}", ["PUT"]).didnt_exist,
endpoint("/state/asset/{name}/{key}", ["DELETE"]).didnt_exist,
endpoint("/state/asset/{name}", ["DELETE"]).didnt_exist,
)
Loading
Loading