Dag Bundles

A Dag bundle is a collection of one or more Dags, files along with their associated files, such as other Python scripts, configuration files, or other resources. Dag bundles can source the Dags from various locations, such as local directories, Git repositories, or other external systems. Deployment administrators can also write their own Dag bundle classes to support custom sources. You can also define more than one Dag bundle in an Airflow deployments, allowing for better organization of your Dags. By keeping the bundle at a higher level, it allows for versioning everything the Dag needs to run.

This is similar, but more powerful than the Dags folder in Airflow 2 or earlier, where Dags were required to be in one place on the local disk, and getting the Dags there was solely the responsibility of the deployment manager.

Since Dag bundles support versioning, they also allow Airflow to run a task using a specific version of the Dag bundle, allowing for a Dag run to use the same code for the whole run, even if the Dag is updated mid-way through the run.

Why are Dag bundles important?

  • Version Control: By supporting versioning, Dag bundles allow Dag runs to use the same code for the whole run, even if the Dag is updated mid way through the run.

  • Scalability: With Dag bundles, Airflow can efficiently manage large numbers of Dags by organizing them into logical units.

  • Flexibility: Dag bundles enable seamless integration with external systems, such as Git repositories, to source Dags.

Types of Dag bundles

Airflow supports multiple types of Dag Bundles, each catering to specific use cases:

airflow.dag_processing.bundles.local.LocalDagBundle

These bundles reference a local directory containing Dag files. They are ideal for development and testing environments, but do not support versioning of the bundle, meaning tasks always run using the latest code.

airflow.providers.git.bundles.git.GitDagBundle

These bundles integrate with Git repositories, allowing Airflow to fetch Dags directly from a repository. The GitDagBundle does support versioning.

airflow.providers.amazon.aws.bundles.s3.S3DagBundle

These bundles reference an S3 bucket containing Dag files. They do not support versioning of the bundle, meaning tasks always run using the latest code.

airflow.providers.google.cloud.bundles.gcs.GCSDagBundle

These bundles reference a GCS bucket containing Dag files. They do not support versioning of the bundle, meaning tasks always run using the latest code.

Configuring Dag bundles

Dag bundles are configured in dag_bundle_config_list. You can add one or more Dag bundles here.

By default, Airflow adds a LocalDagBundle pointing at the configured Dags folder, maintaining the same behaviour as Airflow 2’s Dags folder. The only kwarg is path, which defaults to the value of dags_folder when omitted:

[dag_processor]
dag_bundle_config_list = [
    {
      "name": "dags-folder",
      "classpath": "airflow.dag_processing.bundles.local.LocalDagBundle",
      "kwargs": {
        "path": "/opt/airflow/dags"
      }
    }
  ]

Note

LocalDagBundle does not support versioning. Tasks always run against the latest code on disk.

For a Git Dag bundle, the only required kwarg is tracking_ref (a branch, tag, or commit SHA). Use git_conn_id to reference an Airflow connection that holds the repository credentials, or supply repo_url directly. You can also narrow the checkout to a subdirectory with subdir, or use sparse_dirs to enable a sparse checkout of specific directories:

[dag_processor]
dag_bundle_config_list = [
    {
      "name": "my-git-repo",
      "classpath": "airflow.providers.git.bundles.git.GitDagBundle",
      "kwargs": {
        "git_conn_id": "my_git_conn",
        "subdir": "dags",
        "tracking_ref": "main",
      }
    }
  ]

Note

GitDagBundle supports versioning. Each Dag run records the Git commit it was created with, allowing reruns to use the exact same code even if the repository has since been updated.

See Bundles for the full list of kwargs and more examples.

For an S3 Dag bundle, the required kwarg is bucket_name. You can optionally set aws_conn_id (defaults to aws_default) and prefix to scope the bundle to a subdirectory within the bucket:

[dag_processor]
dag_bundle_config_list = [
    {
      "name": "my-s3-dags",
      "classpath": "airflow.providers.amazon.aws.bundles.s3.S3DagBundle",
      "kwargs": {
        "aws_conn_id": "aws_default",
        "bucket_name": "my-airflow-bucket",
        "prefix": "dags/"
      }
    }
  ]

Note

S3DagBundle does not support versioning. Tasks always run against the latest code in the bucket.

See Bundles for the full list of kwargs and more examples.

For a GCS Dag bundle, the required kwarg is bucket_name. You can optionally set gcp_conn_id (defaults to google_cloud_default) and prefix to scope the bundle to a subdirectory within the bucket:

[dag_processor]
dag_bundle_config_list = [
    {
      "name": "my-gcs-dags",
      "classpath": "airflow.providers.google.cloud.bundles.gcs.GCSDagBundle",
      "kwargs": {
        "gcp_conn_id": "google_cloud_default",
        "bucket_name": "my-airflow-bucket",
        "prefix": "dags/"
      }
    }
  ]

Note

GCSDagBundle does not support versioning. Tasks always run against the latest code in the bucket.

See Bundles for the full list of kwargs and more examples.

You can combine multiple bundle types in a single deployment. The default LocalDagBundle can be removed if you no longer need it, or kept alongside other bundles:

[dag_processor]
dag_bundle_config_list = [
    {
      "name": "my_git_bundle",
      "classpath": "airflow.providers.git.bundles.git.GitDagBundle",
      "kwargs": {"tracking_ref": "main", "git_conn_id": "my_git_conn"}
    },
    {
      "name": "dags-folder",
      "classpath": "airflow.dag_processing.bundles.local.LocalDagBundle",
      "kwargs": {}
    }
  ]

Note

The whitespace, particularly on the last line, is important so a multi-line value works properly. More details can be found in the the configparser docs.

If you want a view url different from the default provided by the Dag bundle, you can change the url in the kwargs of the Dag bundle configuration. For example, if you want to use a custom URL for the git Dag bundle:

[dag_processor]
dag_bundle_config_list = [
    {
      "name": "my_git_repo",
      "classpath": "airflow.providers.git.bundles.git.GitDagBundle",
      "kwargs": {
        "tracking_ref": "main",
        "git_conn_id": "my_git_conn",
        "view_url_template": "https://my.custom.git.repo/view/{subdir}",
      }
    }
  ]

Above, the view_url_template is set to a custom URL that will be used to view the Dags in the my_git_repo bundle. The {subdir} placeholder will be replaced with the subdir attribute of the bundle. The placeholders are attributes of the bundle. You cannot use any placeholder outside of the bundle’s attributes. When you specify a custom URL, it overrides the default URL provided by the Dag bundle.

The url is verified for safety, and if it is not safe, the view url for the bundle will be set to None. This is to prevent any potential security issues with unsafe URLs.

You can also override the refresh_interval per Dag bundle by passing it in kwargs. This controls how often the Dag processor refreshes, or looks for new files, in the Dag bundles.

Starting Airflow 3.0.2 git is pre installed in the base image. However, if you are using versions prior 3.0.2, you would need to install git in your docker image.

RUN apt-get update && apt-get install -y git
ENV GIT_PYTHON_GIT_EXECUTABLE=/usr/bin/git
ENV GIT_PYTHON_REFRESH=quiet

Using DAG Bundles with User Impersonation

When using run_as_user (user impersonation) with DAG bundles, ensure proper file permissions are configured so that impersonated users can access bundle files created by the main Airflow process.

  1. All impersonated users and the Airflow user should be in the same group

  2. Configure appropriate umask settings (e.g., umask 0002)

Note

This permission-based approach is a temporary solution. Future versions of Airflow will handle multi-user access through supervisor-based bundle operations, eliminating the need for shared group permissions.

Configuring Default Rerun Version Behavior

When a user clears a DAG run or task instance, the UI shows a checkbox asking whether to rerun with the latest bundle version or with the version the original run used. The rerun_with_latest_version setting controls the default state of that checkbox, so teams don’t have to make that decision manually every time. The same setting also governs the default run_on_latest_version behavior when creating backfills via the API or CLI.

Note

This only applies to versioned bundle types (like GitDagBundle). Local bundles (LocalDagBundle) do not support versioning and will always use the latest code.

How It Works

Each DAG has a parsed version (DagModel.bundle_version), updated every time the dag processor re-parses the DAG file. Each DAG run records the bundle version it was created with.

When rerun_with_latest_version is False, clearing a DAG run preserves its original bundle version, so the rerun uses the same code. This provides reproducibility when debugging failures. When True, clearing updates the DAG run to the current parsed version, ensuring the most recent code is used on rerun.

The setting is resolved using the following precedence (highest to lowest):

  1. Explicit request: The run_on_latest_version parameter in the API request body (if provided)

  2. DAG-level: The DAG’s rerun_with_latest_version parameter (if True or False)

  3. Global config: The [core] rerun_with_latest_version option (if set)

  4. Per-call-site fallback: False for clear/rerun, True for backfills (preserving the historical default for each path)

Global Configuration

Set organization-wide defaults using the [core] rerun_with_latest_version option:

[core]
rerun_with_latest_version = False  # Rerun with the original bundle version
# rerun_with_latest_version = True  # Rerun with the latest bundle version

When unset, the call site applies the historical fallback (False for clear/rerun, True for backfills).

DAG-Level Configuration

Override the global default for specific DAGs:

from datetime import datetime

from airflow import DAG
from airflow.operators.empty import EmptyOperator

# Always rerun with the latest version
with DAG(
    dag_id="always_latest_dag",
    rerun_with_latest_version=True,
    start_date=datetime(2024, 1, 1),
) as dag:
    EmptyOperator(task_id="task")

Use Cases

Debugging failed runs:

With False (the default), clearing a failed run reruns it with the same code, making it easier to reproduce and isolate issues.

Always run latest code:

Set [core] rerun_with_latest_version = True if your team prefers reruns to always pick up the latest code, for example when bug fixes have been deployed since the original run.

Mixed policy:

Set the global default to True but override specific critical DAGs with rerun_with_latest_version=False for version stability where it matters most.

Relationship with disable_bundle_versioning

Airflow provides two separate settings that affect bundle versioning behavior. They serve different purposes:

disable_bundle_versioning

Turns off version tracking entirely. When set to True, no bundle_version is recorded on DAG runs. Available as a DAG parameter and as a global config option ([dag_processor] disable_bundle_versioning).

rerun_with_latest_version

Controls the default rerun behavior while keeping version tracking active. When a user clears or reruns a task, this determines whether the new run uses the latest bundle version or the original version. Versioning remains enabled so the version history is still recorded. This only changes the default choice presented to users.

In short: disable_bundle_versioning answers “should we track versions at all?”, while rerun_with_latest_version answers “when rerunning, which version should be the default?”. The two settings are independent. rerun_with_latest_version has no effect when versioning is disabled.

Writing custom Dag bundles

When implementing your own Dag bundle by extending the BaseDagBundle class, there are several methods you must implement. Below is a guide to help you implement a custom Dag bundle.

Abstract Methods

The following methods are abstract and must be implemented in your custom bundle class:

path

This property should return a Path to the directory where the Dag files for this bundle are stored. Airflow uses this property to locate the Dag files for processing.

get_current_version

This method should return the current version of the bundle as a string. Airflow will use pass this version to __init__ later to get this version of the bundle again when it runs tasks. If versioning is not supported, it should return None.

refresh

This method should handle refreshing the bundle’s contents from its source (e.g., pulling the latest changes from a remote repository). This is used by the Dag processor periodically to ensure that the bundle is up-to-date.

Optional Methods

In addition to the abstract methods, you may choose to override the following methods to customize the behavior of your bundle:

__init__

This method can be extended to initialize the bundle with extra parameters, such as tracking_ref for the GitDagBundle. It should also call the parent class’s __init__ method to ensure proper initialization. Expensive operations, such as network calls, should be avoided in this method to prevent delays during the bundle’s instantiation, and done in the initialize method instead.

initialize

This method is called before the bundle is first used in the Dag processor or worker. It allows you to perform expensive operations only when the bundle’s content is accessed.

view_url

This method should return a URL as a string to view the bundle on an external system (e.g., a Git repository’s web interface).

Other Considerations

  • Versioning: If your bundle supports versioning, ensure that initialize, get_current_version and refresh are implemented to handle version-specific logic.

  • Concurrency: Workers may create many bundles simultaneously, and does nothing to serialize calls to the bundle objects. Thus, the bundle class must handle locking if that is problematic for the underlying technology. For example, if you are cloning a git repo, the bundle class is responsible for locking to ensure only 1 bundle object is cloning at a time. There is a lock method in the base class that can be used for this purpose, if necessary.

  • Triggerer Limitation: DAG bundles are not initialized in the triggerer component. In practice, this means that triggers cannot come from a DAG bundle. This is because the triggerer does not deal with changes in trigger code over time, as everything happens in the main process. Triggers can come from anywhere else on sys.path instead. If you need to use custom triggers, ensure they are available in the Python environment’s sys.path rather than being sourced from DAG bundles.

Was this entry helpful?