Dagster & dbt (Component)
The dagster-dbt library provides a DbtProjectComponent which can be used to easily represent dbt models as assets in Dagster. Dagster assets understand dbt at the level of individual dbt models. This means that you can:
- Use Dagster's UI or APIs to run subsets of your dbt models, seeds, and snapshots.
- Track failures, logs, and run history for individual dbt models, seeds, and snapshots.
- Define dependencies between individual dbt models and other data assets. For example, put dbt models after the Fivetran-ingested table that they read from, or put a machine learning after the dbt models that it's trained from.
DbtProjectComponent is a state-backed component, which compiles and caches your dbt project's manifest. For information on managing component state, see Configuring state-backed components.
Dagster supports dbt Fusion as of the 1.11.5 release. Dagster will automatically detect which engine you have installed. If you're currently using core, to migrate uninstall dbt-core and install dbt Fusion. For more information please reference the dbt docs.
This feature is still in preview pending dbt Fusion GA.
1. Prepare a Dagster project
To begin, you'll need a Dagster project. You can use an existing components-ready project or create a new one:
create-dagster project my-project && cd my-project
Activate the project virtual environment:
source .venv/bin/activate
Then, add the dagster-dbt library to the project, along with a duckdb adapter:
- uv
- pip
uv add dagster-dbt dbt-duckdb
pip install dagster-dbt dbt-duckdb
2. Set up a dbt project
For this tutorial, we'll use the jaffle shop dbt project as an example. Clone it into your project:
git clone --depth=1 https://github.com/dbt-labs/jaffle_shop.git dbt && rm -rf dbt/.git
We will create a profiles.yml file in the dbt directory to configure the project to use DuckDB:
jaffle_shop:
target: dev
outputs:
dev:
type: duckdb
path: ~/tutorial.duckdb
threads: 24
3. Scaffold a dbt component definition
Now that you have a Dagster project with a dbt project, you can scaffold a dbt component definition. You'll need to provide the path to your dbt project:
dg scaffold defs dagster_dbt.DbtProjectComponent dbt_ingest \
--project-path "dbt"
Creating defs at /.../my-project/src/my_project/defs/dbt_ingest.
The dg scaffold defs call will generate a defs.yaml file in your project structure:
tree src/my_project
src/my_project
├── __init__.py
├── definitions.py
└ ── defs
├── __init__.py
└── dbt_ingest
└── defs.yaml
3 directories, 4 files
In its scaffolded form, the defs.yaml file contains the configuration for your dbt project:
type: dagster_dbt.DbtProjectComponent
attributes:
project: '{{ project_root }}/dbt'
This is sufficient to load your dbt models as assets. You can use dg list defs to see the asset representation:
dg list defs
┏━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓
┃ Section ┃ Definitions ┃
┡━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩
│ Assets │ ┏━━━━━━━━━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━┳━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ │
│ │ ┃ Key ┃ Group ┃ Deps ┃ Kinds ┃ Description ┃ │
│ │ ┡━━━━━━━━━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━╇━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ │
│ │ │ customers │ default │ stg_customers │ dbt │ This table has basic information about a │ │
│ │ │ │ │ stg_orders │ duckdb │ customer, as well as some derived facts based │ │
│ │ │ │ │ stg_payments │ │ on a custome… │ │
│ │ ├───────────────┼─────────┼───────────────┼────────┼────────────────────────────────────────────────┤ │
│ │ │ orders │ default │ stg_orders │ dbt │ This table has basic information about orders, │ │
│ │ │ │ │ stg_payments │ duckdb │ as well as some derived facts based on │ │
│ │ │ │ │ │ │ payments │ │
│ │ │ │ │ │ │ │ │
│ │ │ │ │ │ │ #### … │ │
│ │ ├───────────────┼─────────┼───────────────┼────────┼────────────────────────────────────────────────┤ │
│ │ │ raw_customers │ default │ │ dbt │ dbt seed raw_customers │ │
│ │ │ │ │ │ duckdb │ │ │
│ │ │ │ │ │ │ #### Raw SQL: │ │
│ │ │ │ │ │ │ ```sql │ │
│ │ │ │ │ │ │ │ │
│ │ │ │ │ │ │ ``` │ │
│ │ ├───────────────┼─────────┼───────────────┼────────┼────────────────────────────────────────────────┤ │
│ │ │ raw_orders │ default │ │ dbt │ dbt seed raw_orders │ │
│ │ │ │ │ │ duckdb │ │ │
│ │ │ │ │ │ │ #### Raw SQL: │ │
│ │ │ │ │ │ │ ```sql │ │
│ │ │ │ │ │ │ │ │
│ │ │ │ │ │ │ ``` │ │
│ │ ├───────────────┼─────────┼───────────────┼────────┼────────────────────────────────────────────────┤ │
│ │ │ raw_payments │ default │ │ dbt │ dbt seed raw_payments │ │
│ │ │ │ │ │ duckdb │ │ │
│ │ │ │ │ │ │ #### Raw SQL: │ │
│ │ │ │ │ │ │ ```sql │ │
│ │ │ │ │ │ │ │ │
│ │ │ │ │ │ │ ``` │ │
│ │ ├───────────────┼─────────┼───────────────┼────────┼────────────────────────────────────────────────┤ │
│ │ │ stg_customers │ default │ raw_customers │ dbt │ dbt model stg_customers │ │
│ │ │ │ │ │ duckdb │ │ │
│ │ │ │ │ │ │ #### Raw SQL: │ │
│ │ │ │ │ │ │ ```sql │ │
│ │ │ │ │ │ │ with source as ( │ │
│ │ │ │ │ │ │ │ │
│ │ │ │ │ │ │ {#- │ │
│ │ │ │ │ │ │ Normally we would … │ │
│ │ ├───────────────┼─────────┼───────────────┼────────┼────────────────────────────────────────────────┤ │
│ │ │ stg_orders │ default │ raw_orders │ dbt │ dbt model stg_orders │ │
│ │ │ │ │ │ duckdb │ │ │
│ │ │ │ │ │ │ #### Raw SQL: │ │
│ │ │ │ │ │ │ ```sql │ │
│ │ │ │ │ │ │ with source as ( │ │
│ │ │ │ │ │ │ │ │
│ │ │ │ │ │ │ {#- │ │
│ │ │ │ │ │ │ Normally we would sel… │ │
│ │ ├───────────────┼─────────┼───────────────┼────────┼────────────────────────────────────────────────┤ │
│ │ │ stg_payments │ default │ raw_payments │ dbt │ dbt model stg_payments │ │
│ │ │ │ │ │ duckdb │ │ │
│ │ │ │ │ │ │ #### Raw SQL: │ │
│ │ │ │ │ │ │ ```sql │ │
│ │ │ │ │ │ │ with source as ( │ │
│ │ │ │ │ │ │ │ │
│ │ │ │ │ │ │ {#- │ │
│ │ │ │ │ │ │ Normally we wou… │ │
│ │ └───────────────┴─────────┴───────────────┴────────┴────────────────────────────────────────────────┘ │
│ Asset Checks │ ┏━━━━━━━━━━━ ━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┳┳┓ │
│ │ ┃ Key ┃┃┃ │
│ │ ┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━╇╇┩ │
│ │ │ customers:not_null_customers_customer_id │││ │
│ │ ├─────────────────────────────────────────────────────────────────────────────────────────────────┼┼┤ │
│ │ │ customers:unique_customers_customer_id │││ │
│ │ ├─────────────────────────────────────────────── ──────────────────────────────────────────────────┼┼┤ │
│ │ │ orders:accepted_values_orders_status__placed__shipped__completed__return_pending__returned │││ │
│ │ ├─────────────────────────────────────────────────────────────────────────────────────────────────┼┼┤ │
│ │ │ orders:not_null_orders_amount │││ │
│ │ ├─────────────────────────────────────────────────────────────────────────────────────────────────┼┼┤ │
│ │ │ orders:not_null_orders_bank_transfer_amount │││ │
│ │ ├─────────────────────────────────────────────────────────────────────────────────── ──────────────┼┼┤ │
│ │ │ orders:not_null_orders_coupon_amount │││ │
│ │ ├─────────────────────────────────────────────────────────────────────────────────────────────────┼┼┤ │
│ │ │ orders:not_null_orders_credit_card_amount │││ │
│ │ ├─────────────────────────────────────────────────────────────────────────────────────────────────┼┼┤ │
│ │ │ orders:not_null_orders_customer_id │││ │
│ │ ├─────────────────────────────────────────────────────────────────────────────────────────────────┼┼┤ │
│ │ │ orders:not_null_orders_gift_card_amount │││ │
│ │ ├─────────────────────────────────────────────────────────────────────────────────────────────────┼┼┤ │
│ │ │ orders:not_null_orders_order_id │││ │
│ │ ├─────────────────────────────────────────────────────────────────────────────────────────────────┼┼┤ │
│ │ │ orders:relationships_orders_customer_id__customer_id__ref_customers_ │││ │
│ │ ├─────────────────────────────────────────────────────────────────────────────────────────────────┼┼┤ │
│ │ │ orders:unique_orders_order_id │││ │
│ │ ├─────────────────────────────────────────────────────────────────────────────────────────────────┼┼┤ │
│ │ │ stg_customers:not_null_stg_customers_customer_id │││ │
│ │ ├─────────────────────────────────────────────────────────────────────────────────────────────────┼┼┤ │
│ │ │ stg_customers:unique_stg_customers_customer_id │││ │
│ │ ├─────────────────────────────────────────────────────────────────────────────────────────────────┼┼┤ │
│ │ │ stg_orders:accepted_values_stg_orders_status__placed__shipped__completed__return_pending__retu… │││ │
│ │ ├─────────────────────────────────────────────────────────────────────────────────────────────────┼┼┤ │
│ │ │ stg_orders:not_null_stg_orders_order_id │││ │
│ │ ├─────────────────────────────────────────────────────────────────────────────────────────────────┼┼┤ │
│ │ │ stg_orders:unique_stg_orders_order_id │││ │
│ │ ├─────────────────────────────────────────────────────────────────────────────────────────────────┼┼┤ │
│ │ │ stg_payments:accepted_values_stg_payments_payment_method__credit_card__coupon__bank_transfer__… │││ │
│ │ ├─────────────────────────────────────────────────────────────────────────────────────────────────┼┼┤ │
│ │ │ stg_payments:not_null_stg_payments_payment_id │││ │
│ │ ├─────────────────────────────────────────────────────────────────────────────────────────────────┼┼┤ │
│ │ │ stg_payments:unique_stg_payments_payment_id │││ │
│ │ └─────────────────────────────────────────────────────────────────────────────────────────────────┴┴┘ │
└──────────────┴───────────────────────────────────────────────────────────────────────────────────────────────────────┘
4. Run your dbt models
To execute your dbt models, you can use the dg launch command to kick off a run through the CLI:
dg launch --assets '*'
You can also pass an asset selection to the dg launch --assets command:
dg launch --assets "key:'customers' and key:'orders'"
5. Select or exclude specific models
You can control which dbt models are included in your component using the select or exclude attributes. This allows you to filter which models are represented as assets, using dbt's selection syntax. For example, to include only the customers model:
type: dagster_dbt.DbtProjectComponent
attributes:
project: '{{ project_root }}/dbt'
select: "customers"
dg list defs
┏━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓
┃ Section ┃ Definitions ┃
┡━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩
│ Assets │ ┏━━━━━━━━━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━┳━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ │
│ │ ┃ Key ┃ Group ┃ Deps ┃ Kinds ┃ Description ┃ │
│ │ ┡━━━━━━━━━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━╇━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ │
│ │ │ customers │ default │ stg_customers │ dbt │ This table has basic information about a │ │
│ │ │ │ │ stg_orders │ duckdb │ customer, as well as some derived facts based │ │
│ │ │ │ │ stg_payments │ │ on a custome… │ │
│ │ ├───────────────┼─────────┼───────────────┼────────┼────────────────────────────────────────────────┤ │
│ │ │ stg_customers │ default │ │ │ │ │
│ │ ├───────────────┼─────────┼───────────────┼────────┼────────────────────────────────────────────────┤ │
│ │ │ stg_orders │ default │ │ │ │ │
│ │ ├───────────────┼─────────┼───────────────┼────────┼────────────────────────────────────────────────┤ │
│ │ │ stg_payments │ default │ │ │ │ │
│ │ └───────────────┴─────────┴───────────────┴────────┴────────────────────────────────────────────────┘ │
│ Asset Checks │ ┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━┳━━━━━━━━━━━━━┓ │
│ │ ┃ Key ┃ Deps ┃ Description ┃ │
│ │ ┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━╇━━━━━━━━━━━━━┩ │
│ │ │ customers:not_null_customers_customer_id │ customers │ │ │
│ │ ├──────────────────────────────────────────┼───────────┼─────────────┤ │
│ │ │ customers:unique_customers_customer_id │ customers │ │ │
│ │ └──────────────────────────────────────────┴───────────┴─────────────┘ │
└──────────────┴───────────────────────────────────────────────────────────────────────────────────────────────────────┘
6. Customize dbt assets
You can customize the properties of the assets emitted by each dbt model using the translation key in your defs.yaml file. This allows you to modify asset metadata such as group names, descriptions, and other properties:
type: dagster_dbt.DbtProjectComponent
attributes:
project: '{{ project_root }}/dbt'
select: "customers"
translation:
group_name: dbt_models
description: "Transforms data using dbt model {{ node.name }}"
dg list defs
┏━━━━━━━━━━━━ ━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓
┃ Section ┃ Definitions ┃
┡━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩
│ Assets │ ┏━━━━━━━━━━━━━━━┳━━━━━━━━━━━━┳━━━━━━━━━━━━━━━┳━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ │
│ │ ┃ Key ┃ Group ┃ Deps ┃ Kinds ┃ Description ┃ │
│ │ ┡━━━━━━━━━━━━━━━╇━━━━━━━━━━━━╇━━━━━━━━━━━━━━━╇━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ │
│ │ │ customers │ dbt_models │ stg_customers │ dbt │ Transforms data using dbt model customers │ │
│ │ │ │ │ stg_orders │ duckdb │ │ │
│ │ │ │ │ stg_payments │ │ │ │
│ │ ├───────────────┼────────────┼───────────────┼────────┼───────────────────────────────────────────┤ │
│ │ │ stg_customers │ default │ │ │ │ │
│ │ ├───────────────┼────────────┼───────────────┼────────┼───────────────────────────────────────────┤ │
│ │ │ stg_orders │ default │ │ │ │ │
│ │ ├───────────────┼────────────┼───────────────┼────────┼───────────────────────────────────────────┤ │
│ │ │ stg_payments │ default │ │ │ │ │
│ │ └───────────────┴────────────┴───────────────┴────────┴───────────────────────────────────────────┘ │
│ Asset Checks │ ┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━┳━━━━━━━━━━━━━┓ │
│ │ ┃ Key ┃ Deps ┃ Description ┃ │
│ │ ┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━╇━━━━━━━━━━━━━┩ │
│ │ │ customers:not_null_customers_customer_id │ customers │ │ │
│ │ ├──────────────────────────────────────────┼───────────┼─────────────┤ │
│ │ │ customers:unique_customers_customer_id │ customers │ │ │
│ │ └──────────────────────────────────────────┴───────────┴─────────────┘ │
└──────────────┴─────────────────────────────────────────────────────────────────────────────────────────────────────┘
7. Depending on dbt assets in other components
If you want to refer to assets built by the dbt component elsewhere in your Dagster project, you can use the asset_key_for_model method on the dbt component.
This lets you refer to an asset by the model name without having to know how that model is translated to an asset key.
Imagine a PythonScriptComponent that exports the customers model to a CSV file:
dg scaffold defs dagster.PythonScriptComponent my_python_script
Creating defs at /.../my-project/src/my_project/defs/my_python_script.
touch src/my_project/defs/my_python_script/export_customers.py
You can refer to the customers asset in this component by using the asset_key_for_model method on the dbt component:
type: dagster.PythonScriptComponent
attributes:
execution:
path: export_customers.py
assets:
- key: customers_export
deps:
- "{{ load_component_at_path('dbt_ingest').asset_key_for_model('customers') }}"
dg list defs
┏━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓
┃ Section ┃ Definitions ┃
┡━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩
│ Assets │ ┏━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━┳━━━━━━━━━━━━━━━┳━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ │
│ │ ┃ Key ┃ Group ┃ Deps ┃ Kinds ┃ Description ┃ │
│ │ ┡━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━╇━━━━━━━━━━━━━━━╇━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ │
│ │ │ customers │ dbt_models │ stg_customers │ dbt │ Transforms data using dbt model │ │
│ │ │ │ │ stg_orders │ duckdb │ customers │ │
│ │ │ │ │ stg_payments │ │ │ │
│ │ ├──────────────────┼────────────┼───────────────┼────────┼──────────────────────────────────────────┤ │
│ │ │ customers_export │ default │ customers │ │ │ │
│ │ ├──────────────────┼────────────┼───────────────┼────────┼──────────────────────────────────────────┤ │
│ │ │ stg_customers │ default │ │ │ │ │
│ │ ├──────────────────┼────────────┼───────────────┼────────┼──────────────────────────────────────────┤ │
│ │ │ stg_orders │ default │ │ │ │ │
│ │ ├──────────────────┼────────────┼───────────────┼────────┼──────────────────────────────────────────┤ │
│ │ │ stg_payments │ default │ │ │ │ │
│ │ └──────────────────┴────────────┴───────────────┴────────┴──────────────────────────────────────────┘ │
│ Asset Checks │ ┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━┳━━━━━━━━━━━━━┓ │
│ │ ┃ Key ┃ Deps ┃ Description ┃ │
│ │ ┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━╇━━━━━━━━━━━━━┩ │
│ │ │ customers:not_null_customers_customer_id │ customers │ │ │
│ │ ├──────────────────────────────────────────┼───────────┼─────────────┤ │
│ │ │ customers:unique_customers_customer_id │ customers │ │ │
│ │ └──────────────────────────────────────────┴───────────┴─────────────┘ │
└──────────────┴───────────────────────────────────────────────────────────────────────────────────────────────────────┘
8. Handling incremental models
If you have incremental models in your dbt project, you can model these as partitioned assets, and update the command that is used to run the dbt models to pass in --vars based on the range of partitions that are being processed.
The first step is to add a new template var to your component. This will be used to define the partitions definition that will be used to partition the assets.
import dagster as dg
@dg.template_var
def daily_partitions_def() -> dg.DailyPartitionsDefinition:
return dg.DailyPartitionsDefinition(start_date="2023-01-01")
The next step is to update the defs.yaml file to use the new template var and apply this partitions definition to all assets using the post_process field:
type: dagster_dbt.DbtProjectComponent
template_vars_module: .template_vars
attributes:
project: '{{ project_root }}/dbt'
select: "customers"
translation:
group_name: dbt_models
description: "Transforms data using dbt model {{ node.name }}"
post_processing:
assets:
- target: "*"
attributes:
partitions_def: "{{ daily_partitions_def }}"
Finally, we need to pass in new configuration to the cli_args field so that the dbt execution actually changes based on what partition is executing. In particular, we want to pass in values to the --vars configuration field that determine the range of time that our incremental models should process.
The specific format of this configuration depends on your specific dbt project setup, but one common pattern is to use a start_date and end_date parameter for this purpose.
When the cli_args field is resolved, it has access to a context.partition_time_window object, which is Dagster's representation of the time range that should be processed on the current run. This can be converted into a format recognized by your dbt project using template variables:
type: dagster_dbt.DbtProjectComponent
template_vars_module: .template_vars
attributes:
project: '{{ project_root }}/dbt'
select: "customers"
translation:
group_name: dbt_models
description: "Transforms data using dbt model {{ node.name }}"
cli_args:
- build
- --vars:
start_date: "{{ partition_time_window.start.strftime('%Y-%m-%d') }}"
end_date: "{{ partition_time_window.end.strftime('%Y-%m-%d') }}"
post_processing:
assets:
- target: "*"
attributes:
partitions_def: "{{ daily_partitions_def }}"
Dagster will automatically convert this configuration dictionary into the JSON-encoded string that is expected by the dbt CLI.
dg list defs
┏━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓
┃ Section ┃ Definitions ┃
┡━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩
│ Assets │ ┏━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━┳━━━━━━━━━━━━━━━┳━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ │
│ │ ┃ Key ┃ Group ┃ Deps ┃ Kinds ┃ Description ┃ │
│ │ ┡━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━╇━━━━━━━━━━━━━━━╇━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ │
│ │ │ customers │ dbt_models │ stg_customers │ dbt │ Transforms data using dbt model │ │
│ │ │ │ │ stg_orders │ duckdb │ customers │ │
│ │ │ │ │ stg_payments │ │ │ │
│ │ ├──────────────────┼────────────┼───────────────┼────────┼──────────────────────────────────────────┤ │
│ │ │ customers_export │ default │ customers │ │ │ │
│ │ ├──────────────────┼────────────┼───────────────┼────────┼──────────────────────────────────────────┤ │
│ │ │ stg_customers │ default │ │ │ │ │
│ │ ├──────────────────┼────────────┼───────────────┼────────┼──────────────────────────────────────────┤ │
│ │ │ stg_orders │ default │ │ │ │ │
│ │ ├──────────────────┼────────────┼───────────────┼────────┼──────────────────────────────────────────┤ │
│ │ │ stg_payments │ default │ │ │ │ │
│ │ └──────────────────┴────────────┴───────────────┴────────┴──────────────────────────────────────────┘ │
│ Asset Checks │ ┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━┳━━━━━━━━━━━━━┓ │
│ │ ┃ Key ┃ Deps ┃ Description ┃ │
│ │ ┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━╇━━━━━━━━━━━━━┩ │
│ │ │ customers:not_null_customers_customer_id │ customers │ │ │
│ │ ├──────────────────────────────────────────┼───────────┼─────────────┤ │
│ │ │ customers:unique_customers_customer_id │ customers │ │ │
│ │ └──────────────────────────────────────────┴───────────┴─────────────┘ │
└──────────────┴───────────────────────────────────────────────────────────────────────────────────────────────────────┘
If you have multiple different partitions definitions, you will need to create separate DbtProjectComponent instances for each PartitionsDefinition you want to use. You can filter each component to a selection of dbt models using the select configuration option.
9. Advanced configuration (subclassing)
For more complex use cases that cannot easily be handled with templated yaml, you can create a custom subclass of DbtProjectComponent to add custom behavior. This allows you to:
- Add custom op configuration using
op_config_schema - Override the
get_asset_specmethod to add custom metadata - Override the
executemethod to customize how dbt commands are executed
To do this, we can create a new subclass of the DbtProjectComponent in our lib directory. This component adds a full_refresh runtime config option and custom metadata:
import json
from collections.abc import Iterator, Mapping
from datetime import timedelta
from typing import Any, Optional
from dagster_dbt import DbtCliResource, DbtProject, DbtProjectComponent
import dagster as dg
class CustomDbtProjectComponent(DbtProjectComponent):
"""Custom DbtProjectComponent with op config and metadata customization."""
@property
def op_config_schema(self) -> type[dg.Config]:
class CustomDbtConfig(dg.Config):
full_refresh: bool = False
return CustomDbtConfig
def get_asset_spec(
self, manifest: Mapping[str, Any], unique_id: str, project: Optional[DbtProject]
) -> dg.AssetSpec:
base_spec = super().get_asset_spec(manifest, unique_id, project)
dbt_props = self.get_resource_props(manifest, unique_id)
# Add a custom metadata field with the model name
return base_spec.merge_attributes(
metadata={
"dbt_model_name": dbt_props["name"],
}
)
def execute(
self, context: dg.AssetExecutionContext, dbt: DbtCliResource
) -> Iterator:
dbt_vars = {
# custom time range that includes 3 hours of lookback to ensure we don't miss any data
"min_date": (
context.partition_time_window.start - timedelta(hours=3)
).isoformat(),
"max_date": context.partition_time_window.end.isoformat(),
}
# Build CLI args based on config
args = (
["build", "--full-refresh"]
if context.op_config.get("full_refresh", False)
else ["build", "--vars", json.dumps(dbt_vars)]
)
yield from dbt.cli(args, context=context).stream()
This component can then be swapped in for the standard DbtProjectComponent in our defs.yaml file by updating the type field to reference the new component:
type: my_project.lib.custom_dbt_component.CustomDbtProjectComponent
template_vars_module: .template_vars
attributes:
project: '{{ project_root }}/dbt'
select: "customers"
post_processing:
assets:
- target: "*"
attributes:
partitions_def: "{{ daily_partitions_def }}"
The custom component will be loaded and used just like the standard DbtProjectComponent:
dg list defs
┏━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓
┃ Section ┃ Definitions ┃
┡━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩
│ Assets │ ┏━━━━━━━━━━━━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━┳━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ │
│ │ ┃ Key ┃ Group ┃ Deps ┃ Kinds ┃ Description ┃ │
│ │ ┡━━━━━━━━━━━━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━╇━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ │
│ │ │ customers │ default │ stg_customers │ dbt │ This table has basic information about a │ │
│ │ │ │ │ stg_orders │ duckdb │ customer, as well as some derived facts │ │
│ │ │ │ │ stg_payments │ │ based on a custome… │ │
│ │ ├──────────────────┼─────────┼───────────────┼────────┼─────────────────────────────────────────────┤ │
│ │ │ customers_export │ default │ customers │ │ │ │
│ │ ├──────────────────┼─────────┼───────────────┼────────┼─────────────────────────────────────────────┤ │
│ │ │ stg_customers │ default │ │ │ │ │
│ │ ├──────────────────┼─────────┼─ ──────────────┼────────┼─────────────────────────────────────────────┤ │
│ │ │ stg_orders │ default │ │ │ │ │
│ │ ├──────────────────┼─────────┼───────────────┼────────┼─────────────────────────────────────────────┤ │
│ │ │ stg_payments │ default │ │ │ │ │
│ │ └──────────────────┴─────────┴───────────────┴────────┴─────────────────────────────────────────────┘ │
│ Asset Checks │ ┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━┳━━━━━━━━━━━━━┓ │
│ │ ┃ Key ┃ Deps ┃ Description ┃ │
│ │ ┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━╇━━━━━━━━━━━━━┩ │
│ │ │ customers:not_null_customers_customer_id │ customers │ │ │
│ │ ├──────────────────────────────────────────┼───────────┼─────────────┤ │
│ │ │ customers:unique_customers_customer_id │ customers │ │ │
│ │ └──────────────────────────────────────────┴───────────┴─────────────┘ │
└──────────────┴───────────────────────────────────────────────────────────────────────────────────────────────────────┘
Configuring the custom component at runtime
When you use a custom component with an op_config_schema, you can provide config values at runtime through the Dagster UI or when launching runs from the CLI. For example:
ops:
dbt_injest:
config:
full_refresh: true
This allows you to control the behavior of your dbt runs without modifying the component definition itself.