-
Notifications
You must be signed in to change notification settings - Fork 16.4k
Description
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
- I agree to follow this project's Code of Conduct
Metadata
Metadata
Assignees
Labels
Type
Projects
Status