Skip to main content

Documentation Index

Fetch the complete documentation index at: https://astronomer-preview.mintlify.app/llms.txt

Use this file to discover all available pages before exploring further.

In addition to storing configurations about your Airflow environment, the Airflow metadata database stores data about past and present task runs. Airflow never automatically removes metadata, so the longer you use it, the more task run data is stored in your metadata DB. Over a long enough time, this can result in a bloated metadata DB, which can affect performance across your Airflow environment. When a table in the metadata DB is larger than 50GiB, you might start to experience degraded scheduler performance. This can result in:
  • Slow task scheduling
  • Slow dag parsing
  • Gunicorn timing out when using the Celery executor
  • Slower Airflow UI load times
The following tables in the database are at risk of becoming too large over time:
  • dag_run
  • job
  • log
  • rendered_task_instance_fields
  • task_instance
  • xcom
To keep your Airflow environment running at optimal performance, you can clean the metadata DB using the Airflow CLI airflow db clean command. This command was created as a way to safely clean up your metadata DB without querying it directly. In Airflow 3, this command cannot be called from a Dag because tasks can no longer directly access the metadata DB. Instead you can expose Airflow’s utility function used by the command via an HTTP API using an Airflow Plugin. This tutorial describes how to implement the cleanup Dag and corresponding plugin in Airflow so that you can clean your database using the command directly from the Airflow UI.
Even when using Airflow’s DB clean utilities, deleting data from the metadata database can destroy important data. Read the Warnings section carefully before implementing this tutorial Dag in any production Airflow environment.

Warnings

Deleting data from the metadata database can be an extremely destructive action. If you delete data that future task runs depend on, it’s difficult to restore the database to its previous state without interrupting your data pipelines. Before implementing the Dag in this tutorial, consider the following:
  • When specifying the clean_before_timestamp value, use as old a date as possible. The older the deleted data, the less likely it is to affect your currently running Dags.
  • The Dag in this tutorial drops the archived tables it created in the cleanup process by default using the skip_archive=True argument, and does not maintain any history. If the task fails (for example if it runs for longer than five minutes), the archive tables are not cleared. By calling drop_archived_tables in the second task of the Dag, we ensure all archive tables are dropped even in the event of the first task failing.

Prerequisites

  • An Airflow project
    This Dag has been designed and optimized for Airflow environments running on Astro. Consider adjusting the parameters and code if you’re running the Dag in any other type of Airflow environment.
  • The HTTP Airflow provider installed

Step 1: Create your Dag and plugin

  1. In your dags folder, create a file called db_cleanup.py.
  2. Copy the following code into the Dag file.
    """A DB cleanup dag maintained by Astronomer."""
    
    from datetime import UTC, datetime, timedelta
    
    from airflow.cli.commands.db_command import all_tables
    from airflow.providers.http.hooks.http import HttpHook
    from airflow.sdk import Param, dag, task
    
    
    def get_tables() -> list[str]:
        tables = []
    
        for table in all_tables:
            # can't delete dag versions which may be older than corresponding task instances
            # in order to keep dag_version untouched we also need to ignore the dag table
            # https://github.com/apache/airflow/issues/56192
            if table in {
                "dag_version",
                "dag",
            }:
                continue
            tables.append(table)
    
        return tables
    
    
    @task
    def get_chunked_timestamps(**context) -> list[datetime]:
        from plugins.db_cleanup import OldestTimestampResponse
    
        http_conn_id = context["params"]["http_conn_id"]
        tables = context["params"]["tables"]
        batches = []
    
        response = HttpHook("GET", http_conn_id).run(
            "/db_cleanup/api/oldest_timestamp",
            data={"table_names": tables},
        )
        start_chunk_time = OldestTimestampResponse.model_validate_json(response.content).oldest_timestamp
    
        if start_chunk_time is not None:
            start_ts = start_chunk_time
            end_ts = datetime.fromisoformat(context["params"]["clean_before_timestamp"])
            batch_size_days = context["params"]["batch_size_days"]
    
            while start_ts < end_ts:
                batch_end = min(start_ts + timedelta(days=batch_size_days), end_ts)
                batches.append(batch_end)
                start_ts += timedelta(days=batch_size_days)
        return batches
    
    
    @task(map_index_template="ts {{ clean_before_timestamp }}")
    def db_cleanup(clean_before_timestamp: datetime, **context) -> None:
        context["clean_before_timestamp"] = clean_before_timestamp.isoformat()
        tables = context["params"]["tables"]
        http_conn_id = context["params"]["http_conn_id"]
        HttpHook("DELETE", http_conn_id).run(
            "/db_cleanup/api/records",
            params={
                "clean_before_timestamp": clean_before_timestamp.isoformat(),
                "dry_run": context["params"]["dry_run"],
                "skip_archive": True,
                "table_names": tables,
            },
        )
    
    
    @task(trigger_rule="all_done")
    def clean_archive_tables(**context) -> None:
        tables = context["params"]["tables"]
        http_conn_id = context["params"]["http_conn_id"]
        HttpHook("DELETE", http_conn_id).run(
            "/db_cleanup/api/archived",
            params={"table_names": tables},
        )
    
    
    @dag(
        schedule=None,
        catchup=False,
        description=__doc__,
        doc_md=__doc__,
        render_template_as_native_obj=True,
        max_active_tasks=1,
        max_active_runs=1,
        tags=["astronomer", "cleanup"],
        params={
            "clean_before_timestamp": Param(
                default=(datetime.now(tz=UTC) - timedelta(days=90)).isoformat(),
                type="string",
                format="date-time",
                description="Delete records older than this timestamp. Default is 90 days ago.",
            ),
            "tables": Param(
                default=get_tables(),
                type=["null", "array"],
                examples=get_tables(),
                description="List of tables to clean. Default is all tables.",
            ),
            "dry_run": Param(
                default=False,
                type="boolean",
                description="Show a summary of which tables would be deleted in the api-server logs without actually deleting the records. Default is False.",
            ),
            "batch_size_days": Param(
                default=7,
                type="integer",
                description="Number of days in each batch for the cleanup. Default is 7 days.",
            ),
            "http_conn_id": Param(
                default="http_default",
                type="string",
                description="The HTTP connection ID for calling the cleanup API. Default is 'http_default'.",
            ),
        },
    )
    def astronomer_db_cleanup():
    
        db_cleanup.expand(clean_before_timestamp=get_chunked_timestamps()) >> clean_archive_tables()
    
    
    astronomer_db_cleanup()
    
    Rather than running on a schedule, this Dag is triggered manually by default and includes params so that you’re in full control over how you clean the metadata DB. It includes three tasks:
    • get_chunked_timestamps: creates a list of timestamps to process in batches.
    • db_cleanup: calls the run_cleanup utility.
    • clean_archive_tables: calls the drop_archived_tables utility.
    These three tasks run with params you specify at runtime. The params let you specify:
    • clean_before_timestamp: What age of data to delete. Any data that was created before the specified time will be deleted. The default is to delete all data older than 90 days.
    • tables: Which tables to delete data from. By default all tables supported by the DB cleanup utilities are included except for the dag and dag_version table.
    • dry_run: Whether to run the cleanup as a dry run, meaning that no data is deleted. The dag will instead return the SQL that would be executed based on other parameters you have specified. The default is to run the deletion without a dry run.
    • batch_size_days: What batch size to use in order to cleanup data in batches.
    • http_conn_id: Which HTTP connection to use for calling the API exposing the DB cleanup utilities.
  3. In your plugins folder, create a file called db_cleanup.py.
  4. Copy the following code into the plugin file.
    """A DB cleanup plugin maintained by Astronomer."""
    
    import logging
    import os
    from collections.abc import Generator
    from datetime import datetime
    from typing import Annotated
    
    from airflow.api_fastapi.common.router import AirflowRouter
    from airflow.plugins_manager import AirflowPlugin
    from airflow.utils.db import reflect_tables
    from airflow.utils.db_cleanup import _effective_table_names, drop_archived_tables, run_cleanup
    from airflow.utils.session import create_session
    from fastapi import Depends, FastAPI, Query
    from pydantic import BaseModel
    from sqlalchemy import func
    from sqlalchemy.orm.session import Session
    
    
    def _get_session() -> Generator[Session, None]:
        with create_session() as session:
            yield session
    
    
    logger = logging.getLogger(__name__)
    
    
    class TableInfo(BaseModel):
        table_name: str
        row_estimate: int = 0
        table_bytes: int = 0
        index_bytes: int = 0
        toast_bytes: int = 0
        total_bytes: int = 0
    
    
    class InfoResponse(BaseModel):
        tables: list[TableInfo] = []
    
    
    class OldestTimestampResponse(BaseModel):
        oldest_timestamp: datetime | None = None
    
    
    api = AirflowRouter(tags=["DB API"])
    
    
    @api.get("/info")
    def info(
        *,
        order_by: str = "total_bytes",
        order_desc: bool = True,
        session: Annotated[Session, Depends(_get_session)],
    ) -> InfoResponse:
        """
        Provides information about the size of tables in the metadata database.
        """
        if order_by not in {
            "table_name",
            "row_estimate",
            "table_bytes",
            "index_bytes",
            "toast_bytes",
            "total_bytes",
        }:
            raise ValueError(f"Invalid order_by value: {order_by}")
        query = f"""
        SELECT
            table_name,
            row_estimate,
            total_bytes - index_bytes - COALESCE(toast_bytes, 0) AS table_bytes,
            index_bytes,
            toast_bytes,
            total_bytes
        FROM (
            SELECT
                relname AS table_name,
                c.reltuples::int AS row_estimate,
                pg_indexes_size(c.oid) AS index_bytes,
                pg_total_relation_size(reltoastrelid) AS toast_bytes,
                pg_total_relation_size(c.oid) AS total_bytes
            FROM pg_class c
            LEFT JOIN pg_namespace n ON n.oid = c.relnamespace
            WHERE relkind = 'r'
            AND nspname = :table_schema
        ) a
        ORDER BY {order_by} {"DESC" if order_desc else "ASC"};
        """
        table_schema = "public" if os.getenv("ASTRONOMER_ENVIRONMENT") == "local" else "airflow"
    
        result = session.execute(query, {"table_schema": table_schema})
        response = InfoResponse()
    
        for row in result:
            response.tables.append(TableInfo(**{k: v for k, v in row._mapping.items() if v is not None}))
    
        return response
    
    
    @api.get("/oldest_timestamp")
    def get_oldest_timestamp(
        *,
        table_names: Annotated[list[str] | None, Query()] = None,
        session: Annotated[Session, Depends(_get_session)],
    ) -> OldestTimestampResponse:
        oldest_timestamp_list = []
        existing_tables = reflect_tables(tables=None, session=session).tables
        _, effective_config_dict = _effective_table_names(table_names=table_names)
        for table_name, table_config in effective_config_dict.items():
            if table_name in existing_tables:
                orm_model = table_config.orm_model
                recency_column = table_config.recency_column
                oldest_execution_date = session.query(func.min(recency_column)).select_from(orm_model).scalar()
                if oldest_execution_date:
                    oldest_timestamp_list.append(oldest_execution_date)
                else:
                    logging.info("No data found for %s, skipping...", table_name)
            else:
                logging.warning("Table %s not found. Skipping.", table_name)
    
        response = OldestTimestampResponse()
        if oldest_timestamp_list:
            response.oldest_timestamp = min(oldest_timestamp_list)
        return response
    
    
    @api.delete("/records")
    def delete_records(
        *,
        clean_before_timestamp: datetime,
        table_names: Annotated[list[str] | None, Query()] = None,
        dry_run: bool = False,
        verbose: bool = False,
        skip_archive: bool = False,
        batch_size: int | None = None,
        session: Annotated[Session, Depends(_get_session)],
    ):
        run_cleanup(
            clean_before_timestamp=clean_before_timestamp,
            table_names=table_names,
            dry_run=dry_run,
            verbose=verbose,
            confirm=False,
            skip_archive=skip_archive,
            batch_size=batch_size,
            session=session,
        )
    
    
    @api.delete("/archived")
    def delete_archived(
        *,
        table_names: Annotated[list[str] | None, Query()] = None,
        session: Annotated[Session, Depends(_get_session)],
    ):
        drop_archived_tables(
            table_names=table_names,
            needs_confirm=False,
            session=session,
        )
    
    
    app = FastAPI()
    app.include_router(api, prefix="/api")
    
    
    class AstronomerDBCleanupPlugin(AirflowPlugin):
        name = "AstronomerDBCleanupPlugin"
        fastapi_apps = [
            {
                "app": app,
                "url_prefix": "/db_cleanup",
                "name": "Astronomer DB Cleanup Plugin",
            }
        ]
    
    The plugin exposes the following API endpoints:
    • GET /db_cleanup/api/info: Provide a list of tables with their corresponding sizes and row count estimates. This endpoint is not used by the Dag, but can be useful to get insights into table sizes.
    • GET /db_cleanup/api/oldest_timestamp: Return the oldest timestamp for the tables to cleanup used for calculating batches.
    • DELETE /db_cleanup/api/records: Call the run_cleanup utility.
    • DELETE /db_cleanup/api/archived: Call the drop_archived_tables utility.
Because the DB cleanup utilities are running on the api-server, the corresponding logs will show up in the api-server logs.

Step 2: Configure a HTTP connection

Add a HTTP connection used for calling the API endpoints.
  • host: Set this to the deployment’s URL. For example on Astro this would look like something like https://cmls9yey09fpw01ncvse41m4n.4n.astronomer.run/dse41m4n. When running locally in astro dev this should be set to http://api-server:8080.
  • extra: If needed, set the authorization header. On Astro with an API token this would look something like {"Authorization": "Bearer mytoken1234...abc1234"}.

Step 3: Practice running the Dag

In this step, run the Dag in a local Airflow environment to practice the workflow for cleaning metadata DB records. If you completed Step 1 in your production environment, you will need to repeat it here before starting your local Airflow project. Typically in a fresh local Airflow environment there is not much to clean up. When completing this process in a production environment which has been running for a while, there are more historic records to cleanup.
  1. Run astro dev start in your Astro project to start Airflow, then open the Airflow UI at localhost:8080.
  2. Ensure the Airflow connection http_default with host http://api-server:8080 is set.
    Instead of creating an Airflow connection, you can also define it as an environment variable AIRFLOW_CONN_HTTP_DEFAULT=http://api-server:8080 in your local .env file.
  3. In the Airflow UI, run the astronomer_db_cleanup Dag by clicking the play button and configure the following params:
    • dry_run is enabled
    • Choose an appropriate cutoff date for clean_before_timestamp
  4. Click Trigger.
  5. In a local terminal run astro dev logs --api-server -f to show the api-server logs.
  6. Check that the run_cleanup utility completed successfully. Note that if you created a new Astro project for this tutorial, the run will not show much data to be deleted.
You can now use this Dag to periodically clean data from the Airflow metadata DB as needed.