Skip to content

Execution API / XCom push fails when TaskFlow returns dict with a key containing "/" (slash) in Airflow 3 #55410

@jgoedeke

Description

@jgoedeke

Apache Airflow version

3.0.6

If "Other Airflow 2 version" selected, which one?

No response

What happened?

In Airflow 3 (TaskFlow), returning a dict from a @task where one of the dict KEYS contains a "/" causes the task to fail at the end during XCom push with:

AirflowRuntimeError: API_SERVER_ERROR: {'status_code': 404, 'message': 'Not Found', 'detail': {'detail': 'Not Found'}}.

The same code works when the dict keys do NOT contain "/".

It looks like when @task returns a dict and Airflow treats it as multiple outputs, it creates one XCom per dictionary key. Those per-key XCom writes go through the Execution API and the XCom KEY is placed in the URL path. If the key contains "/", the path is split and the route doesn't match, yielding a 404/4xx.

What you think should happen instead?

The task should succeed and XComs should be written even when dictionary keys contain "/" (either by accepting "/" in the server route, or by safe-encoding/sanitizing keys on client/server consistently). At minimum, the behavior should be documented or validated early with a clear error if "/" in keys is unsupported.

How to reproduce

from airflow.decorators import dag, task

@dag(schedule=None)
def xcom_slash_key_bug():

    @task
    def this_works() -> dict[str, str]:
        return {"Some Key": "Some Value"}
    @task
    def this_works_as_well() -> str:
        return "/some/path/like/key"

    @task
    def this_does_not_work() -> dict[str, str]:
        return {"key with slash /": "Some Value"}

    this_works() >> this_works_as_well() >> this_does_not_work()

dag = xcom_slash_key_bug()

Operating System

Docker image

Versions of Apache Airflow Providers

No response

Deployment

Other Docker-based deployment

Deployment details

No response

Anything else?

Log output:

[2025-09-09, 08:33:14] INFO - Done. Returned value was: {'/some/path/like/key': 'Some Value'}: source="airflow.task.operators.airflow.providers.standard.decorators.python._PythonDecoratedOperator"
[2025-09-09, 08:33:14] INFO - Pushing xcom: ti="RuntimeTaskInstance(id=UUID('01992d2d-836b-716c-b17d-03be4ec4c67a'), task_id='this_does_not_work', dag_id='testing', run_id='manual__2025-09-09T06:33:01.531829+00:00', try_number=1, map_index=-1, hostname='e7e821981d6d', context_carrier={}, task=<Task(_PythonDecoratedOperator): this_does_not_work>, bundle_instance=LocalDagBundle(name=dags-folder), max_tries=0, start_date=datetime.datetime(2025, 9, 9, 6, 33, 11, 94959, tzinfo=TzInfo(UTC)), end_date=None, state=<TaskInstanceState.RUNNING: 'running'>, is_mapped=False, rendered_map_index=None, log_url='https://airflow.damas.pilatus-aircraft.com/dags/testing/runs/manual__2025-09-09T06%3A33%3A01.531829%2B00%3A00/tasks/this_does_not_work?try_number=1')": source="task"
[2025-09-09, 08:33:14] ERROR - Task failed with exception: source="task"
AirflowRuntimeError: API_SERVER_ERROR: {'status_code': 404, 'message': 'Not Found', 'detail': {'detail': 'Not Found'}}
File "/home/airflow/.local/lib/python3.11/site-packages/airflow/sdk/execution_time/task_runner.py", line 931 in run

File "/home/airflow/.local/lib/python3.11/site-packages/airflow/sdk/execution_time/task_runner.py", line 1275 in _push_xcom_if_needed

File "/home/airflow/.local/lib/python3.11/site-packages/airflow/sdk/execution_time/task_runner.py", line 414 in xcom_push

File "/home/airflow/.local/lib/python3.11/site-packages/airflow/sdk/execution_time/task_runner.py", line 561 in _xcom_push

File "/home/airflow/.local/lib/python3.11/site-packages/airflow/sdk/bases/xcom.py", line 84 in set

File "/home/airflow/.local/lib/python3.11/site-packages/airflow/sdk/execution_time/comms.py", line 204 in send

File "/home/airflow/.local/lib/python3.11/site-packages/airflow/sdk/execution_time/comms.py", line 264 in _get_response

File "/home/airflow/.local/lib/python3.11/site-packages/airflow/sdk/execution_time/comms.py", line 251 in _from_frame

Are you willing to submit PR?

  • Yes I am willing to submit a PR!

Code of Conduct

Metadata

Metadata

Assignees

No one assigned

    Labels

    area:APIAirflow's REST/HTTP APIarea:corekind:bugThis is a clearly a bugpriority:mediumBug that should be fixed before next release but would not block a release

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions