If your Airflow DAG contains a single BashOperator("dbt run"), you have lost most of what Airflow gives you. No per-model retries, no lineage in the Airflow UI, no parallelization across workers, no data-aware downstream triggers. Astronomer Cosmos parses manifest.json and emits one Airflow task per dbt node, so your Airflow DAG is your dbt DAG.
This guide shows the pragmatic configuration for a Causeway dbt project running on Astronomer.
What Cosmos gives you
- One Airflow task per dbt model and per test. Retries, logs, and SLAs scope to the individual model.
- The dbt DAG rendered in Airflow's graph view. Lineage matches what you see in
dbt docs. - Data-aware scheduling. Emitting a dataset per model lets downstream DAGs trigger on completion instead of polling schedules.
- Per-project task groups. Multi-project repos coexist cleanly in one Airflow instance.
1. The canonical DAG
from cosmos import DbtDag, ProjectConfig, RenderConfig, ExecutionConfig, ProfileConfig
from cosmos.constants import LoadMode, TestBehavior
from cosmos.profiles import DatabricksTokenProfileMapping
causeway_daily = DbtDag(
project_config=ProjectConfig(
dbt_project_path="/usr/local/airflow/dags/dbt/causeway",
manifest_path="/usr/local/airflow/dags/dbt/causeway/target/manifest.json",
),
render_config=RenderConfig(
load_method=LoadMode.DBT_MANIFEST,
test_behavior=TestBehavior.AFTER_EACH,
emit_datasets=True,
),
execution_config=ExecutionConfig(
dbt_executable_path="/home/astro/.venv/dbt/bin/dbt",
),
profile_config=ProfileConfig(
profile_name="causeway",
target_name="prod",
profile_mapping=DatabricksTokenProfileMapping(
conn_id="databricks_default",
profile_args={"catalog": "prod", "schema": "silver"},
),
),
dag_id="causeway_daily",
schedule="@daily",
)
Every config choice in that block is load-bearing. The next sections explain why.
2. Always use LoadMode.DBT_MANIFEST
Cosmos supports three load methods:
DBT_LS— shells out todbt lswhen the DAG is parsed. Simple, but every Airflow DAG parse runs dbt as a subprocess.DBT_MANIFEST— reads an existingmanifest.jsonfrom disk. Fast.AUTOMATIC— picks one based on what is present.
Warning
Airflow parses DAGs frequently: on scheduler tick, on web UI refresh, on worker startup. With DBT_LS, each parse runs dbt ls, which takes several seconds. In a busy instance you end up running thousands of subprocesses per hour for no purpose. Always use DBT_MANIFEST in production.
The manifest is produced by CI, not by the Airflow instance. Your CI deploy job should:
- Run
dbt parseordbt compileon the project. - Commit or upload the resulting
target/manifest.jsonalongside the DAG bundle. - Let Astronomer's deployment process ship both to the workers.
3. TestBehavior.AFTER_EACH
Three test-behavior modes:
AFTER_EACH— Airflow runs a test task after each model task. A test failure blocks downstream models.AFTER_ALL— one test task at the end. Downstream runs even if tests fail.BUILD— one task perdbt build --select <model>(runs model + tests inline).
AFTER_EACH is the right default: it catches bad data early and halts the blast radius. BUILD is cleaner visually but hides which side failed (run vs. test) in the task log.
Note
TestBehavior.AFTER_EACH does not run every test in the project after every model — only the tests that declared themselves against that model. So a model with three tests emits one "model" task and three test tasks that run after it.
4. emit_datasets=True
Data-aware scheduling lets downstream DAGs trigger on completion of a specific dataset, rather than polling. Cosmos emits a dataset per dbt model when this flag is on:
# Downstream consumer DAG
from airflow.sdk import asset
@asset(schedule=[Dataset("dbt://prod.silver.fct_orders")])
def refresh_executive_report():
...
The dataset URI format is dbt://<catalog>.<schema>.<model_name>.
5. Partial parse is not optional
Cosmos runs each dbt node as a separate subprocess. That is the right design for observability, but the project parse cost is paid per task unless you carry target/partial_parse.msgpack alongside the manifest:
project_config=ProjectConfig(
dbt_project_path="/usr/local/airflow/dags/dbt/causeway",
manifest_path="/usr/local/airflow/dags/dbt/causeway/target/manifest.json",
partial_parse=True,
)
Warning
Without partial_parse=True and a shipped partial_parse.msgpack, a project with 800 models takes its parse cost 800 times per DAG run. Parse cost scales with project size — expect 3 to 15 seconds per task. Ship the msgpack.
6. Separate venv from Airflow
dbt pins are opinionated and will fight Airflow's package constraints. On Astronomer, the standard pattern is a separate virtualenv:
# Dockerfile in the Astronomer project
FROM astrocrpublic.azurecr.io/runtime:11.1.0
COPY dbt-requirements.txt /tmp/
RUN python -m venv /home/astro/.venv/dbt && \
/home/astro/.venv/dbt/bin/pip install -r /tmp/dbt-requirements.txt
# dbt-requirements.txt
dbt-databricks==1.11.0
dbt-core==1.11.0
Then execution_config points Cosmos at the venv's dbt binary (as in the first snippet).
7. Multi-project layout
For repos with more than one dbt project, use DbtTaskGroup per project inside a single DAG:
from airflow import DAG
from cosmos.airflow.task_group import DbtTaskGroup
with DAG("causeway_daily", schedule="@daily") as dag:
finance = DbtTaskGroup(group_id="finance", project_config=...)
product = DbtTaskGroup(group_id="product", project_config=...)
finance >> product
Cross-project ref() via dbt-loom or by passing each project's manifest path as an env var. Cosmos resolves cross-project edges when it renders.
Common mistakes
| Symptom | Root cause |
|---|---|
| DAG parse takes a long time, UI feels slow | LoadMode.DBT_LS or AUTOMATIC falling back to ls. Switch to DBT_MANIFEST. |
| Tests never fail the DAG | TestBehavior.AFTER_ALL with retries set; the final test task is passing because earlier failures were isolated. Use AFTER_EACH. |
| Per-task startup takes 5–15s | No partial_parse.msgpack bundled. Parse is running from scratch per task. |
| Downstream DAGs never trigger | emit_datasets=False (the default). Flip it on. |
Cosmos tries to write logs/ or target/ | Airflow workers are read-only. Set DBT_TARGET_PATH and DBT_LOG_PATH to /tmp/dbt/... in the execution config. |
See also
- Slim CI — how the manifest Cosmos needs gets produced.
- Production readiness — the full checklist, including orchestration items.