Skip to content

Import Errors are not persisted to DB (silent failure) when UnboundLocalError occurs in Git Bundles #60059

@waltervillarreal

Description

@waltervillarreal

Apache Airflow version

3.1.5

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

No response

What happened?

What happened: I am running Airflow 3.1.3 using Docker and Git Bundles to sync DAGs. When I push a DAG file that contains a logical runtime error during parsing (specifically an UnboundLocalError), the following happens:

  1. The DAG disappears from the Airflow UI (Grid/Home).
  2. The "Import Errors" tab/list is empty.
  3. The table import_error in the database is empty.
  4. However, the dag-processor logs clearly show that the error was caught but seemingly failed to persist to the DB.

It seems that specific runtime errors inside the DAG factory prevent the GitDagBundle from correctly mapping the physical file path back to the bundle URI for the database insertion, causing a silent failure in the UI.

What you think should happen instead?

The DAG should appear in the "Import Errors" list in the UI with the stack trace, just like it does for LocalDagBundle or simple SyntaxErrors.

How to reproduce

  1. Configure Airflow 3 with a GitDagBundle.

My dag bundle config:

[dag_processor]
dag_bundle_storage_path = /opt/airflow/dags_bundle

dag_bundle_config_list = [
	  {
      "name": "gitlab_airflow_prod",
      "classpath": "airflow.providers.git.bundles.git.GitDagBundle",
      "kwargs": {
			"tracking_ref": "main", 
            "subdir": "dags",
			"git_conn_id": "gitlab_airflow_prod",
			"view_url_template": "https://localgitlab/airflow/pr-airflow/-/tree/main"
				}
		},
      {
        "name": "dags-folder",
        "classpath": "airflow.dag_processing.bundles.local.LocalDagBundle",
        "kwargs": {}
      }
    ]
refresh_interval = 60
parsing_processes = 2
  1. Push the following DAG to the repository:
from airflow.decorators import dag, task
from datetime import datetime

@dag(
    dag_id="reproduce_unbound_error",
    start_date=datetime(2025, 1, 1),
    schedule=None,
    catchup=False
)
def invalid_dag_logic():
    # Usage before assignment triggers UnboundLocalError at parse time
    print(f"Accessing var: {my_var}")

    @task
    def t1(): pass

    t1()

    my_var = "defined_later"

invalid_dag_logic()
  1. Observe that the DAG does not appear in the UI and no import errors are reported.

Operating System: Linux (Docker)

Deployment: Docker Compose / Official Image

Relevant logs: Logs from dag-processor:

{"timestamp":"2026-01-02T18:11:28.392123Z","level":"info","event":"Filling up the DagBag from /opt/airflow/dags_bundle/gitlab_airflow_prod/tracking_repo/dags/test/test.py","logger":"airflow.models.dagbag.DagBag","filename":"dagbag.py","lineno":593}
{"timestamp":"2026-01-02T18:11:28.416660Z","level":"error","event":"Failed to import: /opt/airflow/dags_bundle/gitlab_airflow_prod/tracking_repo/dags/test/test.py","logger":"airflow.models.dagbag.DagBag","filename":"dagbag.py","lineno":415,"error_detail":[{"exc_type":"UnboundLocalError","exc_value":"cannot access local variable 'variable_problem' where it is not associated with a value","exc_notes":[],"syntax_error":null,"is_cause":false,"frames":[{"filename":"/home/airflow/.local/lib/python3.12/site-packages/airflow/models/dagbag.py","lineno":405,"name":"parse"},{"filename":"<frozen importlib._bootstrap_external>","lineno":999,"name":"exec_module"},{"filename":"<frozen importlib._bootstrap>","lineno":488,"name":"_call_with_frames_removed"},{"filename":"/opt/airflow/dags_bundle/gitlab_airflow_prod/tracking_repo/dags/test/test.py","lineno":31,"name":"<module>"},{"filename":"/home/airflow/.local/lib/python3.12/site-packages/airflow/sdk/definitions/dag.py","lineno":1557,"name":"factory"},{"filename":"/opt/airflow/dags_bundle/gitlab_airflow_prod/tracking_repo/dags/test/test.py","lineno":16,"name":"invalid_dag_logic"}],"is_group":false,"exceptions":[]}]}

Operating System

Red Hat Enterprise Linux release 9.6 (Plow)

Versions of Apache Airflow Providers

apache-airflow-providers-git = 0.0.9

Deployment

Docker-Compose

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

No one assigned

    Labels

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions