kaxil commented on issue #55768: URL: https://github.com/apache/airflow/issues/55768#issuecomment-3353598174
A temporary workaround would be to set [`AIRFLOW__SCHEDULER__NUM_RUNS`](https://airflow.apache.org/docs/apache-airflow/stable/configurations-ref.html#num-runs) to a small value like `100`: ```ini [scheduler] num_runs = 100 ``` This forces the scheduler to exit and restart after 100 scheduling loops, which clears the accumulated memory. If you're running in Kubernetes with proper health probes, the scheduler pod will automatically restart. --- The issue stems from **unbounded growth of the `DBDagBag` cache** inside the scheduler. This wasn't a significant problem in Airflow 2.x, but in 3.x with the introduction of DAG versioning, the problem is amplified: 1. **`DBDagBag._dags` has no size limit**: The cache at [`airflow-core/src/airflow/models/dagbag.py:49`](https://github.com/apache/airflow/blob/964997a7a6da5041fb19e48bc31866ffc6fe7bc7/airflow-core/src/airflow/models/dagbag.py#L49) is an unbounded dictionary that stores deserialized DAGs indefinitely. 2. **DAG versioning multiplies cache entries**: In 3.x, each DAG version gets a unique ID. The cache is keyed by `dag_version_id` (not `dag_id`), meaning: - Every DAG change creates a new version with a new cache entry - For dynamic DAGs that change frequently, this creates a new entry on every parse - Old versions are **never evicted** from the cache - Each `SerializedDAG` object holds full task dictionaries, dependencies, and metadata - This could explain the ~10GB growth you're seeing from the scheduler alone - Especially worse if you use Dynamic dags -- that would lead to more Serialized dag versions. Setting `num_runs` forces a scheduler restart, which clears the Python process memory and resets all caches effectively working around the unbounded cache growth for now until we fix it properly. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
