Under which category would you file this issue?
Airflow Core
Apache Airflow version
3.2.0
What happened and how to reproduce it?
When running a DAG that uses PythonVirtualenvOperator whose callable calls from airflow.sdk import Variable; Variable.get(...), the venv subprocess hangs forever as soon as Variable.get() is called. There is no error, no timeout, no traceback — the airflow dags test / dag.test() invocation simply never returns.
The hang reproduces:
- under
dag.test() (in-process execution)
- under
airflow dags test <dag_id> with a separately running airflow api-server reachable via AIRFLOW__CORE__EXECUTION_API_SERVER_URL
In both cases the last log line from the venv is the user's own print("Python inside virtualenv: 3.10.17 ...") — i.e. the subprocess is alive, the callable has started, and execution blocks on the very next line which is Variable.get(...).
Versions:
- apache-airflow-core==3.2.0
- apache-airflow-providers-standard==1.12.3
- apache-airflow-task-sdk==1.2.0
- Python 3.10.17 (host interpreter and venv interpreter)
- SQLite metadata DB
- OS: Linux
How to reproduce
Minimal script (single file, runs end-to-end):
import json
import os
import shutil
import subprocess
import sys
from datetime import datetime, timezone
from pathlib import Path
AIRFLOW_HOME = Path("/tmp/airflow32_venv_hang")
AIRFLOW_DB = AIRFLOW_HOME / "airflow.db"
DAG_BUNDLE_DIR = AIRFLOW_HOME / "dags_bundle"
AIRFLOW_HOME.mkdir(parents=True, exist_ok=True)
DAG_BUNDLE_DIR.mkdir(parents=True, exist_ok=True)
_THIS_FILE = Path(__file__).resolve()
_BUNDLE_FILE = (DAG_BUNDLE_DIR / "venv_hang_repro.py").resolve()
if _THIS_FILE != _BUNDLE_FILE:
shutil.copy(_THIS_FILE, _BUNDLE_FILE)
os.environ["AIRFLOW_HOME"] = str(AIRFLOW_HOME)
os.environ["AIRFLOW__DATABASE__SQL_ALCHEMY_CONN"] = f"sqlite:///{AIRFLOW_DB}"
os.environ["AIRFLOW__CORE__LOAD_EXAMPLES"] = "False"
os.environ["AIRFLOW__DAG_PROCESSOR__DAG_BUNDLE_CONFIG_LIST"] = json.dumps([{
"name": "dags-folder",
"classpath": "airflow.dag_processing.bundles.local.LocalDagBundle",
"kwargs": {"path": str(DAG_BUNDLE_DIR), "refresh_interval": 0},
}])
os.environ["AIRFLOW_VAR_DEMO_MESSAGE"] = "hello from env"
from airflow.sdk import DAG
from airflow.providers.standard.operators.python import PythonVirtualenvOperator
def read_variable_in_venv():
import sys
from airflow.sdk import Variable
print(f"Python inside virtualenv: {sys.version}", flush=True)
# HANGS HERE FOREVER, no error, no timeout:
value = Variable.get("demo_message", default="default value")
print(f"demo_message = {value}", flush=True)
with DAG(
dag_id="venv_hang_repro",
start_date=datetime(2024, 1, 1, tzinfo=timezone.utc),
schedule=None,
catchup=False,
) as dag:
PythonVirtualenvOperator(
task_id="read_variable_from_airflow",
python_callable=read_variable_in_venv,
python_version="3.10",
requirements=[
"apache-airflow==3.2.0",
],
system_site_packages=True,
)
if __name__ == "__main__":
subprocess.run([sys.executable, "-m", "airflow", "db", "migrate"], check=True)
subprocess.run([sys.executable, "-m", "airflow", "dags", "reserialize"], check=True)
dag.test()
Run: python repro.py. Observed behaviour: last log line is Python inside virtualenv: 3.10.17 ..., then the process hangs indefinitely and has to be killed with Ctrl-C.
What you think should happen instead?
One of:
- Variable.get() inside a venv subprocess should work (return the value / default), or
- fail fast with a clear error.
Operating System
No response
Deployment
None
Apache Airflow Provider(s)
No response
Versions of Apache Airflow Providers
No response
Official Helm Chart version
Not Applicable
Kubernetes Version
No response
Helm Chart configuration
No response
Docker Image customizations
No response
Anything else?
No response
Are you willing to submit PR?
Code of Conduct
Under which category would you file this issue?
Airflow Core
Apache Airflow version
3.2.0
What happened and how to reproduce it?
When running a DAG that uses
PythonVirtualenvOperatorwhose callable callsfrom airflow.sdk import Variable; Variable.get(...), the venv subprocess hangs forever as soon asVariable.get()is called. There is no error, no timeout, no traceback — theairflow dags test / dag.test()invocation simply never returns.The hang reproduces:
dag.test()(in-process execution)airflow dags test <dag_id>with a separately running airflow api-server reachable viaAIRFLOW__CORE__EXECUTION_API_SERVER_URLIn both cases the last log line from the venv is the user's own
print("Python inside virtualenv: 3.10.17 ...")— i.e. the subprocess is alive, the callable has started, and execution blocks on the very next line which is Variable.get(...).Versions:
How to reproduce
Minimal script (single file, runs end-to-end):
Run: python repro.py. Observed behaviour: last log line is Python inside virtualenv: 3.10.17 ..., then the process hangs indefinitely and has to be killed with Ctrl-C.
What you think should happen instead?
One of:
Operating System
No response
Deployment
None
Apache Airflow Provider(s)
No response
Versions of Apache Airflow Providers
No response
Official Helm Chart version
Not Applicable
Kubernetes Version
No response
Helm Chart configuration
No response
Docker Image customizations
No response
Anything else?
No response
Are you willing to submit PR?
Code of Conduct