Skip to content

TriggerDagRunOperator is failing for reason 'Direct database access via the ORM is not allowed in Airflow 3.0' #47499

@atul-astronomer

Description

@atul-astronomer

Apache Airflow version

3.0.0b2

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

No response

What happened?

TriggerDagRunOperator is trying to connect to the DB when 'wait_for_completion' is true.

[2025-03-07, 12:54:58] - Task failed with exception logger="task" error_detail=[{"exc_type":"RuntimeError","exc_value":"Direct database access via the ORM is not allowed in Airflow 3.0","syntax_error":null,"is_cause":false,"frames":[{"filename":"/opt/airflow/task_sdk/src/airflow/sdk/execution_time/task_runner.py","lineno":609,"name":"run"},{"filename":"/opt/airflow/task_sdk/src/airflow/sdk/execution_time/task_runner.py","lineno":730,"name":"_execute_task"},{"filename":"/opt/airflow/airflow/models/baseoperator.py","lineno":168,"name":"wrapper"},{"filename":"/opt/airflow/providers/standard/src/airflow/providers/standard/operators/trigger_dagrun.py","lineno":207,"name":"execute"},{"filename":"/opt/airflow/airflow/utils/session.py","lineno":100,"name":"wrapper"},{"filename":"/usr/local/lib/python3.9/contextlib.py","lineno":119,"name":"enter"},{"filename":"/opt/airflow/airflow/utils/session.py","lineno":40,"name":"create_session"},{"filename":"/opt/airflow/task_sdk/src/airflow/sdk/execution_time/supervisor.py","lineno":207,"name":"init"}]}]

[2025-03-07, 12:54:58] - Top level error logger="task" error_detail=[{"exc_type":"RuntimeError","exc_value":"Direct database access via the ORM is not allowed in Airflow 3.0","syntax_error":null,"is_cause":false,"frames":[{"filename":"/opt/airflow/task_sdk/src/airflow/sdk/execution_time/task_runner.py","lineno":817,"name":"main"},{"filename":"/opt/airflow/task_sdk/src/airflow/sdk/execution_time/task_runner.py","lineno":786,"name":"finalize"},{"filename":"/opt/airflow/providers/standard/src/airflow/providers/standard/operators/trigger_dagrun.py","lineno":80,"name":"get_link"},{"filename":"/opt/airflow/airflow/utils/session.py","lineno":100,"name":"wrapper"},{"filename":"/usr/local/lib/python3.9/contextlib.py","lineno":119,"name":"enter"},{"filename":"/opt/airflow/airflow/utils/session.py","lineno":40,"name":"create_session"},{"filename":"/opt/airflow/task_sdk/src/airflow/sdk/execution_time/supervisor.py","lineno":207,"name":"init"}]}]

What you think should happen instead?

A DAG utilising TriggerDagRunOperator should pass.

How to reproduce

Run the below DAG in AF3 beta2:

Controller DAG:

from airflow import DAG
from airflow.operators.trigger_dagrun import TriggerDagRunOperator
from pendulum import today

dag = DAG(
    dag_id="trigger_controller_dag",
    default_args={"owner": "airflow", "start_date": today('UTC').add(days=-2)},
    schedule=None,
    tags=["core"],
)


trigger = TriggerDagRunOperator(
    task_id="test_trigger_dagrun",
    trigger_dag_id="trigger_target_dag",
    reset_dag_run=True,
    wait_for_completion=True,
    conf={"message": "Hello World"},
    dag=dag,
)

Target DAG:

from airflow.models import DAG
from airflow.providers.standard.operators.bash import BashOperator
from airflow.providers.standard.operators.python import PythonOperator
from pendulum import today


dag = DAG(
    dag_id="trigger_target_dag",
    default_args={"start_date": today('UTC').add(days=-2), "owner": "Airflow"},
    tags=["core"],
    schedule=None,  # This must be none so it's triggered by the controller
    is_paused_upon_creation=False, # This must be set so other workers can pick this dag up. mabye it's a bug idk
)


def run_this_func(**context):
    print(
        f"Remotely received value of {context['dag_run'].conf['message']} for key=message "
    )


run_this = PythonOperator(
    task_id="run_this",
    python_callable=run_this_func,
    dag=dag,
)

# You can also access the DagRun object in templates
bash_task = BashOperator(
    task_id="bash_task",
    bash_command='echo "Here is the message: $message"',
    env={"message": '{{ dag_run.conf["message"] if dag_run else "" }}'},
    dag=dag,
)

Operating System

Linux

Versions of Apache Airflow Providers

No response

Deployment

Other

Deployment details

No response

Anything else?

No response

Are you willing to submit PR?

  • Yes I am willing to submit a PR!

Code of Conduct

Metadata

Metadata

Assignees

Labels

area:corekind:bugThis is a clearly a bugpriority:highHigh priority bug that should be patched quickly but does not require immediate new release

Type

No type

Projects

Status

Done

Milestone

Relationships

None yet

Development

No branches or pull requests

Issue actions